From 120e5b892d5396f10ceb5d957de01c8d74a2e781 Mon Sep 17 00:00:00 2001
From: Seunghwa Kang <45857425+seunghwak@users.noreply.github.com>
Date: Tue, 12 Mar 2024 20:10:13 -0700
Subject: [PATCH 1/5] Enable edge masking in the remaining primitives (#4186)

This PR will update all the remaining primitives to support edge masking.

This PR pulls updates from https://github.com/rapidsai/cugraph/pull/4126 and better be reviewed/merged after PR https://github.com/rapidsai/cugraph/pull/4126.

Authors:
  - Seunghwa Kang (https://github.com/seunghwak)

Approvers:
  - Chuck Hastings (https://github.com/ChuckHastings)
  - Naim (https://github.com/naimnv)
  - Joseph Nke (https://github.com/jnke2016)

URL: https://github.com/rapidsai/cugraph/pull/4186
---
 cpp/include/cugraph/graph_view.hpp            |   4 +-
 cpp/include/cugraph/utilities/misc_utils.cuh  |   8 +
 cpp/src/prims/detail/nbr_intersection.cuh     |  11 +-
 ...m_reduce_dst_key_aggregated_outgoing_e.cuh | 529 +++++++++++-----
 ...t_nbr_intersection_of_e_endpoints_by_v.cuh |  24 +-
 .../transform_reduce_e_by_src_dst_key.cuh     | 292 +++++++--
 cpp/tests/CMakeLists.txt                      |  33 +-
 ...rm_reduce_dst_key_aggregated_outgoing_e.cu | 599 ++++++++++++++++++
 ..._v_transform_reduce_incoming_outgoing_e.cu |  84 +--
 cpp/tests/prims/mg_reduce_v.cu                |  47 +-
 ...st_nbr_intersection_of_e_endpoints_by_v.cu | 289 +++++++++
 cpp/tests/prims/mg_transform_reduce_e.cu      |  41 +-
 .../mg_transform_reduce_e_by_src_dst_key.cu   | 495 +++++++++++++++
 cpp/tests/prims/mg_transform_reduce_v.cu      |  47 +-
 cpp/tests/prims/result_compare.cuh            | 143 +++++
 15 files changed, 2189 insertions(+), 457 deletions(-)
 create mode 100644 cpp/tests/prims/mg_per_v_transform_reduce_dst_key_aggregated_outgoing_e.cu
 create mode 100644 cpp/tests/prims/mg_transform_reduce_dst_nbr_intersection_of_e_endpoints_by_v.cu
 create mode 100644 cpp/tests/prims/mg_transform_reduce_e_by_src_dst_key.cu
 create mode 100644 cpp/tests/prims/result_compare.cuh

diff --git a/cpp/include/cugraph/graph_view.hpp b/cpp/include/cugraph/graph_view.hpp
index 3f3514179bf..cbb52ef3b1e 100644
--- a/cpp/include/cugraph/graph_view.hpp
+++ b/cpp/include/cugraph/graph_view.hpp
@@ -613,7 +613,7 @@ class graph_view_t<vertex_t, edge_t, store_transposed, multi_gpu, std::enable_if
       major_value_range_start_offset);
   }
 
-  // FIXME: deprecated, replaced with copmute_number_of_edges (which works with or without edge
+  // FIXME: deprecated, replaced with compute_number_of_edges (which works with or without edge
   // masking)
   edge_t number_of_edges() const
   {
@@ -923,7 +923,7 @@ class graph_view_t<vertex_t, edge_t, store_transposed, multi_gpu, std::enable_if
       offsets_, indices_, this->number_of_vertices());
   }
 
-  // FIXME: deprecated, replaced with copmute_number_of_edges (which works with or without edge
+  // FIXME: deprecated, replaced with compute_number_of_edges (which works with or without edge
   // masking)
   edge_t number_of_edges() const
   {
diff --git a/cpp/include/cugraph/utilities/misc_utils.cuh b/cpp/include/cugraph/utilities/misc_utils.cuh
index d3917a3e851..633dabe5b40 100644
--- a/cpp/include/cugraph/utilities/misc_utils.cuh
+++ b/cpp/include/cugraph/utilities/misc_utils.cuh
@@ -94,6 +94,14 @@ thrust::optional<T> to_thrust_optional(std::optional<T> val)
   return ret;
 }
 
+template <typename T>
+std::optional<T> to_std_optional(thrust::optional<T> val)
+{
+  std::optional<T> ret{std::nullopt};
+  if (val) { ret = *val; }
+  return ret;
+}
+
 template <typename idx_t, typename offset_t>
 rmm::device_uvector<idx_t> expand_sparse_offsets(raft::device_span<offset_t const> offsets,
                                                  idx_t base_idx,
diff --git a/cpp/src/prims/detail/nbr_intersection.cuh b/cpp/src/prims/detail/nbr_intersection.cuh
index e0a04eb59da..847c1db6937 100644
--- a/cpp/src/prims/detail/nbr_intersection.cuh
+++ b/cpp/src/prims/detail/nbr_intersection.cuh
@@ -50,6 +50,7 @@
 #include <thrust/iterator/transform_iterator.h>
 #include <thrust/iterator/zip_iterator.h>
 #include <thrust/optional.h>
+#include <thrust/reduce.h>
 #include <thrust/remove.h>
 #include <thrust/scan.h>
 #include <thrust/set_operations.h>
@@ -1232,9 +1233,11 @@ nbr_intersection(raft::handle_t const& handle,
           rx_v_pair_nbr_intersection_sizes.size() + 1, handle.get_stream());
         rx_v_pair_nbr_intersection_offsets.set_element_to_zero_async(size_t{0},
                                                                      handle.get_stream());
+        auto size_first = thrust::make_transform_iterator(
+          rx_v_pair_nbr_intersection_sizes.begin(), cugraph::detail::typecast_t<edge_t, size_t>{});
         thrust::inclusive_scan(handle.get_thrust_policy(),
-                               rx_v_pair_nbr_intersection_sizes.begin(),
-                               rx_v_pair_nbr_intersection_sizes.end(),
+                               size_first,
+                               size_first + rx_v_pair_nbr_intersection_sizes.size(),
                                rx_v_pair_nbr_intersection_offsets.begin() + 1);
 
         rx_v_pair_nbr_intersection_indices.resize(
@@ -1344,8 +1347,8 @@ nbr_intersection(raft::handle_t const& handle,
         }
 
         thrust::inclusive_scan(handle.get_thrust_policy(),
-                               rx_v_pair_nbr_intersection_sizes.begin(),
-                               rx_v_pair_nbr_intersection_sizes.end(),
+                               size_first,
+                               size_first + rx_v_pair_nbr_intersection_sizes.size(),
                                rx_v_pair_nbr_intersection_offsets.begin() + 1);
 
         std::vector<size_t> h_rx_v_pair_lasts(rx_v_pair_counts.size());
diff --git a/cpp/src/prims/per_v_transform_reduce_dst_key_aggregated_outgoing_e.cuh b/cpp/src/prims/per_v_transform_reduce_dst_key_aggregated_outgoing_e.cuh
index 3b25ae50773..5e4cd81513e 100644
--- a/cpp/src/prims/per_v_transform_reduce_dst_key_aggregated_outgoing_e.cuh
+++ b/cpp/src/prims/per_v_transform_reduce_dst_key_aggregated_outgoing_e.cuh
@@ -16,6 +16,7 @@
 #pragma once
 
 #include "detail/graph_partition_utils.cuh"
+#include "prims/detail/optional_dataframe_buffer.hpp"
 #include "prims/kv_store.cuh"
 #include "utilities/collect_comm.cuh"
 
@@ -83,15 +84,23 @@ struct rebase_offset_t {
 
 // a workaround for cudaErrorInvalidDeviceFunction error when device lambda is used
 template <typename vertex_t, typename edge_value_t>
-struct triplet_to_minor_comm_rank_t {
+struct tuple_to_minor_comm_rank_t {
   compute_vertex_partition_id_from_ext_vertex_t<vertex_t> key_func{};
   int minor_comm_size{};
 
-  __device__ int operator()(
+  template <typename edge_value_type = edge_value_t>
+  __device__ std::enable_if_t<!std::is_same_v<edge_value_type, thrust::nullopt_t>, int> operator()(
     thrust::tuple<vertex_t, vertex_t, edge_value_t> val /* major, minor key, edge value */) const
   {
     return key_func(thrust::get<1>(val)) % minor_comm_size;
   }
+
+  template <typename edge_value_type = edge_value_t>
+  __device__ std::enable_if_t<std::is_same_v<edge_value_type, thrust::nullopt_t>, int> operator()(
+    thrust::tuple<vertex_t, vertex_t> val /* major, minor key */) const
+  {
+    return key_func(thrust::get<1>(val)) % minor_comm_size;
+  }
 };
 
 // a workaround for cudaErrorInvalidDeviceFunction error when device lambda is used
@@ -106,6 +115,7 @@ struct pair_to_binary_partition_id_t {
 // a workaround for cudaErrorInvalidDeviceFunction error when device lambda is used
 template <typename vertex_t,
           typename edge_value_t,
+          typename e_op_result_t,
           typename EdgePartitionDeviceView,
           typename EdgeMajorValueMap,
           typename EdgePartitionMajorValueInputWrapper,
@@ -118,8 +128,10 @@ struct call_key_aggregated_e_op_t {
   EdgeMinorKeyValueMap edge_minor_key_value_map{};
   KeyAggregatedEdgeOp key_aggregated_e_op{};
 
-  __device__ auto operator()(thrust::tuple<vertex_t, vertex_t, edge_value_t>
-                               val /* major, minor key, aggregated edge value */) const
+  template <typename edge_value_type = edge_value_t>
+  __device__ std::enable_if_t<!std::is_same_v<edge_value_type, thrust::nullopt_t>, e_op_result_t>
+  operator()(thrust::tuple<vertex_t, vertex_t, edge_value_t>
+               val /* major, minor key, aggregated edge value */) const
   {
     auto major                 = thrust::get<0>(val);
     auto minor_key             = thrust::get<1>(val);
@@ -131,6 +143,20 @@ struct call_key_aggregated_e_op_t {
     return key_aggregated_e_op(
       major, minor_key, major_val, edge_minor_key_value_map.find(minor_key), aggregated_edge_value);
   }
+
+  template <typename edge_value_type = edge_value_t>
+  __device__ std::enable_if_t<std::is_same_v<edge_value_type, thrust::nullopt_t>, e_op_result_t>
+  operator()(thrust::tuple<vertex_t, vertex_t> val /* major, minor key */) const
+  {
+    auto major     = thrust::get<0>(val);
+    auto minor_key = thrust::get<1>(val);
+    auto major_val = edge_major_value_map
+                       ? (*edge_major_value_map).find(major)
+                       : edge_partition_major_value_input.get(
+                           edge_partition.major_offset_from_major_nocheck(major));
+    return key_aggregated_e_op(
+      major, minor_key, major_val, edge_minor_key_value_map.find(minor_key), thrust::nullopt);
+  }
 };
 
 // a workaround for cudaErrorInvalidDeviceFunction error when device lambda is used
@@ -182,9 +208,8 @@ struct reduce_with_init_t {
  * @tparam EdgeSrcValueInputWrapper Type of the wrapper for edge source property values.
  * @tparam EdgeValueInputWrapper Type of the wrapper for edge property values.
  * @tparam EdgeDstKeyInputWrapper Type of the wrapper for edge destination key values.
- * @tparam VertexIterator Type of the iterator for keys in (key, value) pairs (key type should
- * coincide with vertex type).
- * @tparam ValueIterator Type of the iterator for values in (key, value) pairs.
+ * @tparam KVStoreViewType Type of the (key, value) store. Key type should coincide with vertex
+ * type.
  * @tparam KeyAggregatedEdgeOp Type of the quinary key-aggregated edge operator.
  * @tparam ReduceOp Type of the binary reduction operator.
  * @tparam T Type of the initial value for per-vertex reduction.
@@ -204,15 +229,10 @@ struct reduce_with_init_t {
  * @param edge_dst_key_input Wrapper used to access destination input key values (for the edge
  * destinations assigned to this process in multi-GPU). Use  cugraph::edge_dst_property_t::view().
  * Use update_edge_dst_property to fill the wrapper.
- * @param map_unique_key_first Iterator pointing to the first (inclusive) key in (key, value) pairs
- * (assigned to this process in multi-GPU, `cugraph::detail::compute_gpu_id_from_ext_vertex_t` is
- * used to map keys to processes). (Key, value) pairs may be provided by
- * transform_reduce_by_src_key_e() or transform_reduce_by_dst_key_e().
- * @param map_unique_key_last Iterator pointing to the last (exclusive) key in (key, value) pairs
- * (assigned to this process in multi-GPU).
- * @param map_value_first Iterator pointing to the first (inclusive) value in (key, value) pairs
- * (assigned to this process in multi-GPU). `map_value_last` (exclusive) is deduced as @p
- * map_value_first + thrust::distance(@p map_unique_key_first, @p map_unique_key_last).
+ * @param kv_store_view view object of the (key, value) store (for the keys assigned to this process
+ * in multi-GPU). `cugraph::detail::compute_gpu_id_from_ext_vertex_t` is used to map keys to
+ * processes). (Key, value) pairs may be provided by transform_reduce_e_by_src_key() or
+ * transform_reduce_e_by_dst_key().
  * @param key_aggregated_e_op Quinary operator takes 1) edge source, 2) key, 3) *(@p
  * edge_partition_src_value_input_first + i), 4) value for the key stored in the input (key, value)
  * pairs provided by @p map_unique_key_first, @p map_unique_key_last, and @p map_value_first
@@ -263,8 +283,11 @@ void per_v_transform_reduce_dst_key_aggregated_outgoing_e(
   using edge_src_value_t = typename EdgeSrcValueInputWrapper::value_type;
   using edge_value_t     = typename EdgeValueInputWrapper::value_type;
   using kv_pair_value_t  = typename KVStoreViewType::value_type;
+  using optional_edge_value_buffer_value_type =
+    std::conditional_t<!std::is_same_v<edge_value_t, thrust::nullopt_t>, edge_value_t, void>;
+
   static_assert(
-    std::is_arithmetic_v<edge_value_t>,
+    std::is_same_v<edge_value_t, thrust::nullopt_t> || std::is_arithmetic_v<edge_value_t>,
     "Currently only scalar values are supported, should be extended to support thrust::tuple of "
     "arithmetic types and void (for dummy property values) to be consistent with other "
     "primitives.");  // this will also require a custom edge value aggregation op.
@@ -284,16 +307,15 @@ void per_v_transform_reduce_dst_key_aggregated_outgoing_e(
     detail::edge_partition_edge_dummy_property_device_view_t<vertex_t>,
     detail::edge_partition_edge_property_device_view_t<
       edge_t,
-      typename EdgeValueInputWrapper::value_iterator>>;
-
-  CUGRAPH_EXPECTS(!graph_view.has_edge_mask(), "unimplemented.");
+      typename EdgeValueInputWrapper::value_iterator,
+      typename EdgeValueInputWrapper::value_type>>;
 
   if (do_expensive_check) { /* currently, nothing to do */
   }
 
   auto total_global_mem = handle.get_device_properties().totalGlobalMem;
   size_t element_size   = sizeof(vertex_t) * 2;  // major + minor keys
-  if constexpr (!std::is_same_v<edge_value_t, void>) {
+  if constexpr (!std::is_same_v<edge_value_t, thrust::nullopt_t>) {
     static_assert(is_arithmetic_or_thrust_tuple_of_arithmetic<edge_value_t>::value);
     if constexpr (is_thrust_tuple_of_arithmetic<edge_value_t>::value) {
       element_size += sum_thrust_tuple_element_sizes<edge_value_t>();
@@ -317,24 +339,78 @@ void per_v_transform_reduce_dst_key_aggregated_outgoing_e(
 
   // 1. aggregate each vertex out-going edges based on keys and transform-reduce.
 
+  auto edge_mask_view = graph_view.edge_mask_view();
+
   rmm::device_uvector<vertex_t> majors(0, handle.get_stream());
   auto e_op_result_buffer = allocate_dataframe_buffer<T>(0, handle.get_stream());
   for (size_t i = 0; i < graph_view.number_of_local_edge_partitions(); ++i) {
     auto edge_partition =
       edge_partition_device_view_t<vertex_t, edge_t, GraphViewType::is_multi_gpu>(
         graph_view.local_edge_partition_view(i));
+    auto edge_partition_e_mask =
+      edge_mask_view
+        ? thrust::make_optional<
+            detail::edge_partition_edge_property_device_view_t<edge_t, uint32_t const*, bool>>(
+            *edge_mask_view, i)
+        : thrust::nullopt;
 
     auto edge_partition_src_value_input =
       edge_partition_src_input_device_view_t(edge_src_value_input, i);
     auto edge_partition_e_value_input = edge_partition_e_input_device_view_t(edge_value_input, i);
 
-    rmm::device_uvector<vertex_t> tmp_majors(edge_partition.number_of_edges(), handle.get_stream());
+    std::optional<rmm::device_uvector<edge_t>> offsets_with_mask{std::nullopt};
+    if (edge_partition_e_mask) {
+      rmm::device_uvector<edge_t> degrees_with_mask(0, handle.get_stream());
+      if (edge_partition.dcs_nzd_vertices()) {
+        auto segment_offsets = graph_view.local_edge_partition_segment_offsets(i);
+
+        auto major_sparse_range_size =
+          (*segment_offsets)[detail::num_sparse_segments_per_vertex_partition];
+        degrees_with_mask = rmm::device_uvector<edge_t>(
+          major_sparse_range_size + *(edge_partition.dcs_nzd_vertex_count()), handle.get_stream());
+        auto major_first = thrust::make_transform_iterator(
+          thrust::make_counting_iterator(vertex_t{0}),
+          cuda::proclaim_return_type<vertex_t>(
+            [major_sparse_range_size,
+             major_range_first = edge_partition.major_range_first(),
+             dcs_nzd_vertices  = *(edge_partition.dcs_nzd_vertices())] __device__(vertex_t i) {
+              if (i < major_sparse_range_size) {  // sparse
+                return major_range_first + i;
+              } else {  // hypersparse
+                return *(dcs_nzd_vertices + (i - major_sparse_range_size));
+              }
+            }));
+        degrees_with_mask =
+          edge_partition.compute_local_degrees_with_mask((*edge_partition_e_mask).value_first(),
+                                                         major_first,
+                                                         major_first + degrees_with_mask.size(),
+                                                         handle.get_stream());
+      } else {
+        degrees_with_mask = edge_partition.compute_local_degrees_with_mask(
+          (*edge_partition_e_mask).value_first(),
+          thrust::make_counting_iterator(edge_partition.major_range_first()),
+          thrust::make_counting_iterator(edge_partition.major_range_last()),
+          handle.get_stream());
+      }
+      offsets_with_mask =
+        rmm::device_uvector<edge_t>(degrees_with_mask.size() + 1, handle.get_stream());
+      (*offsets_with_mask).set_element_to_zero_async(0, handle.get_stream());
+      thrust::inclusive_scan(handle.get_thrust_policy(),
+                             degrees_with_mask.begin(),
+                             degrees_with_mask.end(),
+                             (*offsets_with_mask).begin() + 1);
+    }
+
+    rmm::device_uvector<vertex_t> tmp_majors(
+      edge_partition_e_mask ? (*offsets_with_mask).back_element(handle.get_stream())
+                            : edge_partition.number_of_edges(),
+      handle.get_stream());
     rmm::device_uvector<vertex_t> tmp_minor_keys(tmp_majors.size(), handle.get_stream());
-    // FIXME: this doesn't work if edge_value_t is thrust::tuple or void
-    rmm::device_uvector<edge_value_t> tmp_key_aggregated_edge_values(tmp_majors.size(),
-                                                                     handle.get_stream());
+    auto tmp_key_aggregated_edge_values =
+      detail::allocate_optional_dataframe_buffer<optional_edge_value_buffer_value_type>(
+        tmp_majors.size(), handle.get_stream());
 
-    if (edge_partition.number_of_edges() > 0) {
+    if (tmp_majors.size() > 0) {
       auto segment_offsets = graph_view.local_edge_partition_segment_offsets(i);
 
       detail::decompress_edge_partition_to_fill_edgelist_majors<vertex_t,
@@ -342,7 +418,7 @@ void per_v_transform_reduce_dst_key_aggregated_outgoing_e(
                                                                 GraphViewType::is_multi_gpu>(
         handle,
         edge_partition,
-        std::nullopt,
+        detail::to_std_optional(edge_partition_e_mask),
         raft::device_span<vertex_t>(tmp_majors.data(), tmp_majors.size()),
         segment_offsets);
 
@@ -357,14 +433,14 @@ void per_v_transform_reduce_dst_key_aggregated_outgoing_e(
         static_cast<size_t>(handle.get_device_properties().multiProcessorCount) * (1 << 20);
       auto [h_vertex_offsets, h_edge_offsets] = detail::compute_offset_aligned_element_chunks(
         handle,
-        raft::device_span<edge_t const>{
-          edge_partition.offsets(),
-          1 + static_cast<size_t>(
-                edge_partition.dcs_nzd_vertices()
-                  ? (*segment_offsets)[detail::num_sparse_segments_per_vertex_partition] +
-                      *(edge_partition.dcs_nzd_vertex_count())
-                  : edge_partition.major_range_size())},
-        edge_partition.number_of_edges(),
+        raft::device_span<edge_t const>(
+          offsets_with_mask ? (*offsets_with_mask).data() : edge_partition.offsets(),
+          (edge_partition.dcs_nzd_vertices()
+             ? (*segment_offsets)[detail::num_sparse_segments_per_vertex_partition] +
+                 *(edge_partition.dcs_nzd_vertex_count())
+             : edge_partition.major_range_size()) +
+            1),
+        static_cast<edge_t>(tmp_majors.size()),
         approx_edges_to_sort_per_iteration);
       auto num_chunks = h_vertex_offsets.size() - 1;
 
@@ -376,30 +452,69 @@ void per_v_transform_reduce_dst_key_aggregated_outgoing_e(
       rmm::device_uvector<vertex_t> unreduced_majors(max_chunk_size, handle.get_stream());
       rmm::device_uvector<vertex_t> unreduced_minor_keys(unreduced_majors.size(),
                                                          handle.get_stream());
-      // FIXME: this doesn't work if edge_value_t is thrust::tuple or void
-      rmm::device_uvector<edge_value_t> unreduced_key_aggregated_edge_values(
-        unreduced_majors.size(), handle.get_stream());
+      auto unreduced_key_aggregated_edge_values =
+        detail::allocate_optional_dataframe_buffer<optional_edge_value_buffer_value_type>(
+          unreduced_majors.size(), handle.get_stream());
       rmm::device_uvector<std::byte> d_tmp_storage(0, handle.get_stream());
 
       size_t reduced_size{0};
       for (size_t j = 0; j < num_chunks; ++j) {
-        thrust::copy(handle.get_thrust_policy(),
-                     minor_key_first + h_edge_offsets[j],
-                     minor_key_first + h_edge_offsets[j + 1],
-                     tmp_minor_keys.begin() + h_edge_offsets[j]);
+        if (edge_partition_e_mask) {
+          std::array<edge_t, 2> unmasked_ranges{};
+          raft::update_host(unmasked_ranges.data(),
+                            edge_partition.offsets() + h_vertex_offsets[j],
+                            1,
+                            handle.get_stream());
+          raft::update_host(unmasked_ranges.data() + 1,
+                            edge_partition.offsets() + h_vertex_offsets[j + 1],
+                            1,
+                            handle.get_stream());
+          handle.sync_stream();
+          if constexpr (!std::is_same_v<edge_value_t, thrust::nullopt_t>) {
+            detail::copy_if_mask_set(
+              handle,
+              thrust::make_zip_iterator(minor_key_first,
+                                        edge_partition_e_value_input.value_first()) +
+                unmasked_ranges[0],
+              thrust::make_zip_iterator(minor_key_first,
+                                        edge_partition_e_value_input.value_first()) +
+                unmasked_ranges[1],
+              (*edge_partition_e_mask).value_first() + unmasked_ranges[0],
+              thrust::make_zip_iterator(tmp_minor_keys.begin(),
+                                        detail::get_optional_dataframe_buffer_begin<edge_value_t>(
+                                          tmp_key_aggregated_edge_values)) +
+                h_edge_offsets[j]);
+          } else {
+            detail::copy_if_mask_set(handle,
+                                     minor_key_first + unmasked_ranges[0],
+                                     minor_key_first + unmasked_ranges[1],
+                                     (*edge_partition_e_mask).value_first() + unmasked_ranges[0],
+                                     tmp_minor_keys.begin() + h_edge_offsets[j]);
+          }
+        } else {
+          thrust::copy(handle.get_thrust_policy(),
+                       minor_key_first + h_edge_offsets[j],
+                       minor_key_first + h_edge_offsets[j + 1],
+                       tmp_minor_keys.begin() + h_edge_offsets[j]);
+        }
 
         size_t tmp_storage_bytes{0};
-        auto offset_first =
-          thrust::make_transform_iterator(edge_partition.offsets() + h_vertex_offsets[j],
-                                          detail::rebase_offset_t<edge_t>{h_edge_offsets[j]});
-        if constexpr (!std::is_same_v<edge_value_t, void>) {
+        auto offset_first = thrust::make_transform_iterator(
+          (offsets_with_mask ? (*offsets_with_mask).data() : edge_partition.offsets()) +
+            h_vertex_offsets[j],
+          detail::rebase_offset_t<edge_t>{h_edge_offsets[j]});
+        if constexpr (!std::is_same_v<edge_value_t, thrust::nullopt_t>) {
           cub::DeviceSegmentedSort::SortPairs(
             static_cast<void*>(nullptr),
             tmp_storage_bytes,
             tmp_minor_keys.begin() + h_edge_offsets[j],
             unreduced_minor_keys.begin(),
-            edge_partition_e_value_input.value_first() + h_edge_offsets[j],
-            unreduced_key_aggregated_edge_values.begin(),
+            (edge_partition_e_mask ? detail::get_optional_dataframe_buffer_begin<edge_value_t>(
+                                       tmp_key_aggregated_edge_values)
+                                   : edge_partition_e_value_input.value_first()) +
+              h_edge_offsets[j],
+            detail::get_optional_dataframe_buffer_begin<edge_value_t>(
+              unreduced_key_aggregated_edge_values),
             h_edge_offsets[j + 1] - h_edge_offsets[j],
             h_vertex_offsets[j + 1] - h_vertex_offsets[j],
             offset_first,
@@ -419,14 +534,18 @@ void per_v_transform_reduce_dst_key_aggregated_outgoing_e(
         if (tmp_storage_bytes > d_tmp_storage.size()) {
           d_tmp_storage = rmm::device_uvector<std::byte>(tmp_storage_bytes, handle.get_stream());
         }
-        if constexpr (!std::is_same_v<edge_value_t, void>) {
+        if constexpr (!std::is_same_v<edge_value_t, thrust::nullopt_t>) {
           cub::DeviceSegmentedSort::SortPairs(
             d_tmp_storage.data(),
             tmp_storage_bytes,
             tmp_minor_keys.begin() + h_edge_offsets[j],
             unreduced_minor_keys.begin(),
-            edge_partition_e_value_input.value_first() + h_edge_offsets[j],
-            unreduced_key_aggregated_edge_values.begin(),
+            (edge_partition_e_mask ? detail::get_optional_dataframe_buffer_begin<edge_value_t>(
+                                       tmp_key_aggregated_edge_values)
+                                   : edge_partition_e_value_input.value_first()) +
+              h_edge_offsets[j],
+            detail::get_optional_dataframe_buffer_begin<edge_value_t>(
+              unreduced_key_aggregated_edge_values),
             h_edge_offsets[j + 1] - h_edge_offsets[j],
             h_vertex_offsets[j + 1] - h_vertex_offsets[j],
             offset_first,
@@ -448,39 +567,44 @@ void per_v_transform_reduce_dst_key_aggregated_outgoing_e(
                      tmp_majors.begin() + h_edge_offsets[j],
                      tmp_majors.begin() + h_edge_offsets[j + 1],
                      unreduced_majors.begin());
-        auto input_key_first = thrust::make_zip_iterator(
-          thrust::make_tuple(unreduced_majors.begin(), unreduced_minor_keys.begin()));
+        auto input_key_first =
+          thrust::make_zip_iterator(unreduced_majors.begin(), unreduced_minor_keys.begin());
         auto output_key_first =
-          thrust::make_zip_iterator(thrust::make_tuple(tmp_majors.begin(), tmp_minor_keys.begin()));
-        if constexpr (!std::is_same_v<edge_value_t, void>) {
+          thrust::make_zip_iterator(tmp_majors.begin(), tmp_minor_keys.begin());
+        if constexpr (!std::is_same_v<edge_value_t, thrust::nullopt_t>) {
           reduced_size +=
             thrust::distance(output_key_first + reduced_size,
                              thrust::get<0>(thrust::reduce_by_key(
                                handle.get_thrust_policy(),
                                input_key_first,
                                input_key_first + (h_edge_offsets[j + 1] - h_edge_offsets[j]),
-                               unreduced_key_aggregated_edge_values.begin(),
+                               detail::get_optional_dataframe_buffer_begin<edge_value_t>(
+                                 unreduced_key_aggregated_edge_values),
                                output_key_first + reduced_size,
-                               tmp_key_aggregated_edge_values.begin() + reduced_size)));
+                               detail::get_optional_dataframe_buffer_begin<edge_value_t>(
+                                 tmp_key_aggregated_edge_values) +
+                                 reduced_size)));
         } else {
-          reduced_size +=
-            thrust::distance(output_key_first + reduced_size,
-                             thrust::get<0>(thrust::unique(
-                               handle.get_thrust_policy(),
-                               input_key_first,
-                               input_key_first + (h_edge_offsets[j + 1] - h_edge_offsets[j]),
-                               output_key_first + reduced_size)));
+          reduced_size += thrust::distance(
+            output_key_first + reduced_size,
+            thrust::copy_if(
+              handle.get_thrust_policy(),
+              input_key_first,
+              input_key_first + (h_edge_offsets[j + 1] - h_edge_offsets[j]),
+              thrust::make_counting_iterator(size_t{0}),
+              output_key_first + reduced_size,
+              cugraph::detail::is_first_in_run_t<decltype(input_key_first)>{input_key_first}));
         }
       }
       tmp_majors.resize(reduced_size, handle.get_stream());
       tmp_minor_keys.resize(tmp_majors.size(), handle.get_stream());
-      // FIXME: this doesn't work if edge_value_t is thrust::tuple or void
-      tmp_key_aggregated_edge_values.resize(tmp_majors.size(), handle.get_stream());
+      detail::resize_optional_dataframe_buffer<optional_edge_value_buffer_value_type>(
+        tmp_key_aggregated_edge_values, tmp_majors.size(), handle.get_stream());
     }
     tmp_majors.shrink_to_fit(handle.get_stream());
     tmp_minor_keys.shrink_to_fit(handle.get_stream());
-    // FIXME: this doesn't work if edge_value_t is thrust::tuple or void
-    tmp_key_aggregated_edge_values.shrink_to_fit(handle.get_stream());
+    detail::shrink_to_fit_optional_dataframe_buffer<optional_edge_value_buffer_value_type>(
+      tmp_key_aggregated_edge_values, handle.get_stream());
 
     std::unique_ptr<
       kv_store_t<vertex_t,
@@ -499,18 +623,34 @@ void per_v_transform_reduce_dst_key_aggregated_outgoing_e(
       auto& minor_comm = handle.get_subcomm(cugraph::partition_manager::minor_comm_name());
       auto const minor_comm_size = minor_comm.get_size();
 
-      // FIXME: this doesn't work if edge_value_t is thrust::tuple or void
-      auto triplet_first     = thrust::make_zip_iterator(thrust::make_tuple(
-        tmp_majors.begin(), tmp_minor_keys.begin(), tmp_key_aggregated_edge_values.begin()));
-      auto d_tx_value_counts = cugraph::groupby_and_count(
-        triplet_first,
-        triplet_first + tmp_majors.size(),
-        detail::triplet_to_minor_comm_rank_t<vertex_t, edge_value_t>{
-          detail::compute_vertex_partition_id_from_ext_vertex_t<vertex_t>{comm_size},
-          minor_comm_size},
-        minor_comm_size,
-        mem_frugal_threshold,
-        handle.get_stream());
+      rmm::device_uvector<size_t> d_tx_value_counts(0, handle.get_stream());
+      if constexpr (!std::is_same_v<edge_value_t, thrust::nullopt_t>) {
+        auto triplet_first =
+          thrust::make_zip_iterator(tmp_majors.begin(),
+                                    tmp_minor_keys.begin(),
+                                    detail::get_optional_dataframe_buffer_begin<edge_value_t>(
+                                      tmp_key_aggregated_edge_values));
+        d_tx_value_counts = cugraph::groupby_and_count(
+          triplet_first,
+          triplet_first + tmp_majors.size(),
+          detail::tuple_to_minor_comm_rank_t<vertex_t, edge_value_t>{
+            detail::compute_vertex_partition_id_from_ext_vertex_t<vertex_t>{comm_size},
+            minor_comm_size},
+          minor_comm_size,
+          mem_frugal_threshold,
+          handle.get_stream());
+      } else {
+        auto pair_first   = thrust::make_zip_iterator(tmp_majors.begin(), tmp_minor_keys.begin());
+        d_tx_value_counts = cugraph::groupby_and_count(
+          pair_first,
+          pair_first + tmp_majors.size(),
+          detail::tuple_to_minor_comm_rank_t<vertex_t, edge_value_t>{
+            detail::compute_vertex_partition_id_from_ext_vertex_t<vertex_t>{comm_size},
+            minor_comm_size},
+          minor_comm_size,
+          mem_frugal_threshold,
+          handle.get_stream());
+      }
 
       std::vector<size_t> h_tx_value_counts(d_tx_value_counts.size());
       raft::update_host(h_tx_value_counts.data(),
@@ -544,8 +684,7 @@ void per_v_transform_reduce_dst_key_aggregated_outgoing_e(
           thrust::copy(
             handle.get_thrust_policy(), tmp_majors.begin(), tmp_majors.end(), majors.begin());
 
-          auto pair_first =
-            thrust::make_zip_iterator(thrust::make_tuple(minor_comm_ranks.begin(), majors.begin()));
+          auto pair_first = thrust::make_zip_iterator(minor_comm_ranks.begin(), majors.begin());
           thrust::sort(
             handle.get_thrust_policy(), pair_first, pair_first + minor_comm_ranks.size());
           auto unique_pair_last = thrust::unique(
@@ -622,7 +761,9 @@ void per_v_transform_reduce_dst_key_aggregated_outgoing_e(
 
       rmm::device_uvector<vertex_t> rx_majors(0, handle.get_stream());
       rmm::device_uvector<vertex_t> rx_minor_keys(0, handle.get_stream());
-      rmm::device_uvector<edge_value_t> rx_key_aggregated_edge_values(0, handle.get_stream());
+      auto rx_key_aggregated_edge_values =
+        detail::allocate_optional_dataframe_buffer<optional_edge_value_buffer_value_type>(
+          0, handle.get_stream());
       auto mem_frugal_flag =
         host_scalar_allreduce(minor_comm,
                               tmp_majors.size() > mem_frugal_threshold ? int{1} : int{0},
@@ -639,66 +780,120 @@ void per_v_transform_reduce_dst_key_aggregated_outgoing_e(
         tmp_minor_keys.resize(0, handle.get_stream());
         tmp_minor_keys.shrink_to_fit(handle.get_stream());
 
-        std::tie(rx_key_aggregated_edge_values, std::ignore) =
-          shuffle_values(minor_comm,
-                         tmp_key_aggregated_edge_values.begin(),
-                         h_tx_value_counts,
-                         handle.get_stream());
-        tmp_key_aggregated_edge_values.resize(0, handle.get_stream());
-        tmp_key_aggregated_edge_values.shrink_to_fit(handle.get_stream());
+        if constexpr (!std::is_same_v<edge_value_t, thrust::nullopt_t>) {
+          std::tie(rx_key_aggregated_edge_values, std::ignore) =
+            shuffle_values(minor_comm,
+                           detail::get_optional_dataframe_buffer_begin<edge_value_t>(
+                             tmp_key_aggregated_edge_values),
+                           h_tx_value_counts,
+                           handle.get_stream());
+        }
+        detail::resize_optional_dataframe_buffer<optional_edge_value_buffer_value_type>(
+          tmp_key_aggregated_edge_values, 0, handle.get_stream());
+        detail::shrink_to_fit_optional_dataframe_buffer<optional_edge_value_buffer_value_type>(
+          tmp_key_aggregated_edge_values, handle.get_stream());
       } else {
-        std::forward_as_tuple(std::tie(rx_majors, rx_minor_keys, rx_key_aggregated_edge_values),
-                              std::ignore) =
-          shuffle_values(minor_comm, triplet_first, h_tx_value_counts, handle.get_stream());
+        if constexpr (!std::is_same_v<edge_value_t, thrust::nullopt_t>) {
+          auto triplet_first =
+            thrust::make_zip_iterator(tmp_majors.begin(),
+                                      tmp_minor_keys.begin(),
+                                      detail::get_optional_dataframe_buffer_begin<edge_value_t>(
+                                        tmp_key_aggregated_edge_values));
+          std::forward_as_tuple(std::tie(rx_majors, rx_minor_keys, rx_key_aggregated_edge_values),
+                                std::ignore) =
+            shuffle_values(minor_comm, triplet_first, h_tx_value_counts, handle.get_stream());
+        } else {
+          auto pair_first = thrust::make_zip_iterator(tmp_majors.begin(), tmp_minor_keys.begin());
+          std::forward_as_tuple(std::tie(rx_majors, rx_minor_keys), std::ignore) =
+            shuffle_values(minor_comm, pair_first, h_tx_value_counts, handle.get_stream());
+        }
         tmp_majors.resize(0, handle.get_stream());
         tmp_majors.shrink_to_fit(handle.get_stream());
         tmp_minor_keys.resize(0, handle.get_stream());
         tmp_minor_keys.shrink_to_fit(handle.get_stream());
-        tmp_key_aggregated_edge_values.resize(0, handle.get_stream());
-        tmp_key_aggregated_edge_values.shrink_to_fit(handle.get_stream());
+        detail::resize_optional_dataframe_buffer<optional_edge_value_buffer_value_type>(
+          tmp_key_aggregated_edge_values, 0, handle.get_stream());
+        detail::shrink_to_fit_optional_dataframe_buffer<optional_edge_value_buffer_value_type>(
+          tmp_key_aggregated_edge_values, handle.get_stream());
       }
 
-      auto key_pair_first =
-        thrust::make_zip_iterator(thrust::make_tuple(rx_majors.begin(), rx_minor_keys.begin()));
-      if (rx_majors.size() > mem_frugal_threshold) {  // trade-off parallelism to lower peak memory
-        auto second_first =
-          detail::mem_frugal_partition(key_pair_first,
-                                       key_pair_first + rx_majors.size(),
-                                       rx_key_aggregated_edge_values.begin(),
-                                       detail::pair_to_binary_partition_id_t<vertex_t>{},
-                                       int{1},
-                                       handle.get_stream());
-
-        thrust::sort_by_key(handle.get_thrust_policy(),
-                            key_pair_first,
-                            std::get<0>(second_first),
-                            rx_key_aggregated_edge_values.begin());
-
-        thrust::sort_by_key(handle.get_thrust_policy(),
-                            std::get<0>(second_first),
-                            key_pair_first + rx_majors.size(),
-                            std::get<1>(second_first));
+      auto key_pair_first = thrust::make_zip_iterator(rx_majors.begin(), rx_minor_keys.begin());
+      if constexpr (!std::is_same_v<edge_value_t, thrust::nullopt_t>) {
+        if (rx_majors.size() >
+            mem_frugal_threshold) {  // trade-off parallelism to lower peak memory
+          auto second_first =
+            detail::mem_frugal_partition(key_pair_first,
+                                         key_pair_first + rx_majors.size(),
+                                         detail::get_optional_dataframe_buffer_begin<edge_value_t>(
+                                           rx_key_aggregated_edge_values),
+                                         detail::pair_to_binary_partition_id_t<vertex_t>{},
+                                         int{1},
+                                         handle.get_stream());
+
+          thrust::sort_by_key(handle.get_thrust_policy(),
+                              key_pair_first,
+                              std::get<0>(second_first),
+                              detail::get_optional_dataframe_buffer_begin<edge_value_t>(
+                                rx_key_aggregated_edge_values));
+
+          thrust::sort_by_key(handle.get_thrust_policy(),
+                              std::get<0>(second_first),
+                              key_pair_first + rx_majors.size(),
+                              std::get<1>(second_first));
+        } else {
+          thrust::sort_by_key(handle.get_thrust_policy(),
+                              key_pair_first,
+                              key_pair_first + rx_majors.size(),
+                              detail::get_optional_dataframe_buffer_begin<edge_value_t>(
+                                rx_key_aggregated_edge_values));
+        }
+
+        auto num_uniques =
+          thrust::count_if(handle.get_thrust_policy(),
+                           thrust::make_counting_iterator(size_t{0}),
+                           thrust::make_counting_iterator(rx_majors.size()),
+                           detail::is_first_in_run_t<decltype(key_pair_first)>{key_pair_first});
+        tmp_majors.resize(num_uniques, handle.get_stream());
+        tmp_minor_keys.resize(tmp_majors.size(), handle.get_stream());
+        detail::resize_optional_dataframe_buffer<edge_value_t>(
+          tmp_key_aggregated_edge_values, tmp_majors.size(), handle.get_stream());
+        thrust::reduce_by_key(
+          handle.get_thrust_policy(),
+          key_pair_first,
+          key_pair_first + rx_majors.size(),
+          detail::get_optional_dataframe_buffer_begin<edge_value_t>(rx_key_aggregated_edge_values),
+          thrust::make_zip_iterator(tmp_majors.begin(), tmp_minor_keys.begin()),
+          detail::get_optional_dataframe_buffer_begin<edge_value_t>(
+            tmp_key_aggregated_edge_values));
       } else {
-        thrust::sort_by_key(handle.get_thrust_policy(),
-                            key_pair_first,
-                            key_pair_first + rx_majors.size(),
-                            rx_key_aggregated_edge_values.begin());
+        if (rx_majors.size() >
+            mem_frugal_threshold) {  // trade-off parallelism to lower peak memory
+          auto second_first =
+            detail::mem_frugal_partition(key_pair_first,
+                                         key_pair_first + rx_majors.size(),
+                                         detail::pair_to_binary_partition_id_t<vertex_t>{},
+                                         int{1},
+                                         handle.get_stream());
+
+          thrust::sort(handle.get_thrust_policy(), key_pair_first, second_first);
+
+          thrust::sort(handle.get_thrust_policy(), second_first, key_pair_first + rx_majors.size());
+        } else {
+          thrust::sort(
+            handle.get_thrust_policy(), key_pair_first, key_pair_first + rx_majors.size());
+        }
+
+        auto num_uniques = thrust::distance(
+          key_pair_first,
+          thrust::unique(
+            handle.get_thrust_policy(), key_pair_first, key_pair_first + rx_majors.size()));
+        tmp_majors.resize(num_uniques, handle.get_stream());
+        tmp_minor_keys.resize(tmp_majors.size(), handle.get_stream());
+        thrust::copy(handle.get_thrust_policy(),
+                     key_pair_first,
+                     key_pair_first + num_uniques,
+                     thrust::make_zip_iterator(tmp_majors.begin(), tmp_minor_keys.begin()));
       }
-      auto num_uniques =
-        thrust::count_if(handle.get_thrust_policy(),
-                         thrust::make_counting_iterator(size_t{0}),
-                         thrust::make_counting_iterator(rx_majors.size()),
-                         detail::is_first_in_run_t<decltype(key_pair_first)>{key_pair_first});
-      tmp_majors.resize(num_uniques, handle.get_stream());
-      tmp_minor_keys.resize(tmp_majors.size(), handle.get_stream());
-      tmp_key_aggregated_edge_values.resize(tmp_majors.size(), handle.get_stream());
-      thrust::reduce_by_key(
-        handle.get_thrust_policy(),
-        key_pair_first,
-        key_pair_first + rx_majors.size(),
-        rx_key_aggregated_edge_values.begin(),
-        thrust::make_zip_iterator(thrust::make_tuple(tmp_majors.begin(), tmp_minor_keys.begin())),
-        tmp_key_aggregated_edge_values.begin());
     }
 
     std::unique_ptr<kv_store_t<vertex_t, kv_pair_value_t, KVStoreViewType::binary_search>>
@@ -756,8 +951,6 @@ void per_v_transform_reduce_dst_key_aggregated_outgoing_e(
     auto tmp_e_op_result_buffer =
       allocate_dataframe_buffer<T>(tmp_majors.size(), handle.get_stream());
 
-    auto triplet_first = thrust::make_zip_iterator(thrust::make_tuple(
-      tmp_majors.begin(), tmp_minor_keys.begin(), tmp_key_aggregated_edge_values.begin()));
     auto major_value_map_device_view =
       (GraphViewType::is_multi_gpu && edge_src_value_input.keys())
         ? thrust::make_optional<detail::kv_binary_search_store_device_view_t<
@@ -768,28 +961,56 @@ void per_v_transform_reduce_dst_key_aggregated_outgoing_e(
                        detail::kv_cuco_store_find_device_view_t<KVStoreViewType>>
       dst_key_value_map_device_view(
         GraphViewType::is_multi_gpu ? multi_gpu_minor_key_value_map_ptr->view() : kv_store_view);
-    thrust::transform(handle.get_thrust_policy(),
-                      triplet_first,
-                      triplet_first + tmp_majors.size(),
-                      get_dataframe_buffer_begin(tmp_e_op_result_buffer),
-                      detail::call_key_aggregated_e_op_t<
-                        vertex_t,
-                        edge_value_t,
-                        decltype(edge_partition),
-                        std::remove_reference_t<decltype(*major_value_map_device_view)>,
-                        edge_partition_src_input_device_view_t,
-                        decltype(dst_key_value_map_device_view),
-                        KeyAggregatedEdgeOp>{edge_partition,
-                                             major_value_map_device_view,
-                                             edge_partition_src_value_input,
-                                             dst_key_value_map_device_view,
-                                             key_aggregated_e_op});
+    if constexpr (!std::is_same_v<edge_value_t, thrust::nullopt_t>) {
+      auto triplet_first = thrust::make_zip_iterator(
+        tmp_majors.begin(),
+        tmp_minor_keys.begin(),
+        detail::get_optional_dataframe_buffer_begin<edge_value_t>(tmp_key_aggregated_edge_values));
+      thrust::transform(handle.get_thrust_policy(),
+                        triplet_first,
+                        triplet_first + tmp_majors.size(),
+                        get_dataframe_buffer_begin(tmp_e_op_result_buffer),
+                        detail::call_key_aggregated_e_op_t<
+                          vertex_t,
+                          edge_value_t,
+                          T,
+                          decltype(edge_partition),
+                          std::remove_reference_t<decltype(*major_value_map_device_view)>,
+                          edge_partition_src_input_device_view_t,
+                          decltype(dst_key_value_map_device_view),
+                          KeyAggregatedEdgeOp>{edge_partition,
+                                               major_value_map_device_view,
+                                               edge_partition_src_value_input,
+                                               dst_key_value_map_device_view,
+                                               key_aggregated_e_op});
+    } else {
+      auto pair_first = thrust::make_zip_iterator(tmp_majors.begin(), tmp_minor_keys.begin());
+      thrust::transform(handle.get_thrust_policy(),
+                        pair_first,
+                        pair_first + tmp_majors.size(),
+                        get_dataframe_buffer_begin(tmp_e_op_result_buffer),
+                        detail::call_key_aggregated_e_op_t<
+                          vertex_t,
+                          edge_value_t,
+                          T,
+                          decltype(edge_partition),
+                          std::remove_reference_t<decltype(*major_value_map_device_view)>,
+                          edge_partition_src_input_device_view_t,
+                          decltype(dst_key_value_map_device_view),
+                          KeyAggregatedEdgeOp>{edge_partition,
+                                               major_value_map_device_view,
+                                               edge_partition_src_value_input,
+                                               dst_key_value_map_device_view,
+                                               key_aggregated_e_op});
+    }
 
     if constexpr (GraphViewType::is_multi_gpu) { multi_gpu_minor_key_value_map_ptr.reset(); }
     tmp_minor_keys.resize(0, handle.get_stream());
     tmp_minor_keys.shrink_to_fit(handle.get_stream());
-    tmp_key_aggregated_edge_values.resize(0, handle.get_stream());
-    tmp_key_aggregated_edge_values.shrink_to_fit(handle.get_stream());
+    detail::resize_optional_dataframe_buffer<optional_edge_value_buffer_value_type>(
+      tmp_key_aggregated_edge_values, 0, handle.get_stream());
+    detail::shrink_to_fit_optional_dataframe_buffer<optional_edge_value_buffer_value_type>(
+      tmp_key_aggregated_edge_values, handle.get_stream());
 
     {
       auto num_uniques =
diff --git a/cpp/src/prims/transform_reduce_dst_nbr_intersection_of_e_endpoints_by_v.cuh b/cpp/src/prims/transform_reduce_dst_nbr_intersection_of_e_endpoints_by_v.cuh
index b63b014ed05..244586e6d9e 100644
--- a/cpp/src/prims/transform_reduce_dst_nbr_intersection_of_e_endpoints_by_v.cuh
+++ b/cpp/src/prims/transform_reduce_dst_nbr_intersection_of_e_endpoints_by_v.cuh
@@ -19,6 +19,7 @@
 #include "prims/detail/nbr_intersection.cuh"
 #include "prims/property_op_utils.cuh"
 
+#include <cugraph/detail/decompress_edge_partition.cuh>
 #include <cugraph/edge_partition_device_view.cuh>
 #include <cugraph/edge_partition_endpoint_property_device_view.cuh>
 #include <cugraph/edge_src_dst_property.hpp>
@@ -130,7 +131,9 @@ std::tuple<rmm::device_uvector<vertex_t>, ValueBuffer> sort_and_reduce_by_vertic
                         vertices.end(),
                         get_dataframe_buffer_begin(value_buffer),
                         reduced_vertices.begin(),
-                        get_dataframe_buffer_begin(reduced_value_buffer));
+                        get_dataframe_buffer_begin(reduced_value_buffer),
+                        thrust::equal_to<vertex_t>{},
+                        property_op<value_t, thrust::plus>{});
 
   vertices.resize(size_t{0}, handle.get_stream());
   resize_dataframe_buffer(value_buffer, size_t{0}, handle.get_stream());
@@ -201,14 +204,14 @@ struct accumulate_vertex_property_t {
  * @param graph_view Non-owning graph object.
  * @param edge_src_value_input Wrapper used to access source input property values (for the edge
  * sources assigned to this process in multi-GPU). Use either cugraph::edge_src_property_t::view()
- * (if @p e_op needs to access source property values) or cugraph::edge_src_dummy_property_t::view()
- * (if @p e_op does not access source property values). Use update_edge_src_property to fill the
- * wrapper.
+ * (if @p intersection_op needs to access source property values) or
+ * cugraph::edge_src_dummy_property_t::view() (if @p intersection_op does not access source property
+ * values). Use update_edge_src_property to fill the wrapper.
  * @param edge_dst_value_input Wrapper used to access destination input property values (for the
  * edge destinations assigned to this process in multi-GPU). Use either
- * cugraph::edge_dst_property_t::view() (if @p e_op needs to access destination property values) or
- * cugraph::edge_dst_dummy_property_t::view() (if @p e_op does not access destination property
- * values). Use update_edge_dst_property to fill the wrapper.
+ * cugraph::edge_dst_property_t::view() (if @p intersection_op needs to access destination property
+ * values) or cugraph::edge_dst_dummy_property_t::view() (if @p intersection_op does not access
+ * destination property values). Use update_edge_dst_property to fill the wrapper.
  * @param intersection_op quinary operator takes edge source, edge destination, property values for
  * the source, property values for the destination, and a list of vertices in the intersection of
  * edge source & destination vertices' destination neighbors and returns a thrust::tuple of three
@@ -260,8 +263,6 @@ void transform_reduce_dst_nbr_intersection_of_e_endpoints_by_v(
       typename EdgeDstValueInputWrapper::value_iterator,
       typename EdgeDstValueInputWrapper::value_type>>;
 
-  CUGRAPH_EXPECTS(!graph_view.has_edge_mask(), "unimplemented.");
-
   if (do_expensive_check) {
     // currently, nothing to do.
   }
@@ -272,6 +273,7 @@ void transform_reduce_dst_nbr_intersection_of_e_endpoints_by_v(
                init);
 
   auto edge_mask_view = graph_view.edge_mask_view();
+
   for (size_t i = 0; i < graph_view.number_of_local_edge_partitions(); ++i) {
     auto edge_partition =
       edge_partition_device_view_t<vertex_t, edge_t, GraphViewType::is_multi_gpu>(
@@ -484,7 +486,9 @@ void transform_reduce_dst_nbr_intersection_of_e_endpoints_by_v(
                             merged_vertices.end(),
                             get_dataframe_buffer_begin(merged_value_buffer),
                             reduced_vertices.begin(),
-                            get_dataframe_buffer_begin(reduced_value_buffer));
+                            get_dataframe_buffer_begin(reduced_value_buffer),
+                            thrust::equal_to<vertex_t>{},
+                            property_op<T, thrust::plus>{});
       merged_vertices.resize(size_t{0}, handle.get_stream());
       merged_vertices.shrink_to_fit(handle.get_stream());
       resize_dataframe_buffer(merged_value_buffer, size_t{0}, handle.get_stream());
diff --git a/cpp/src/prims/transform_reduce_e_by_src_dst_key.cuh b/cpp/src/prims/transform_reduce_e_by_src_dst_key.cuh
index eee0ed03d1c..00876012906 100644
--- a/cpp/src/prims/transform_reduce_e_by_src_dst_key.cuh
+++ b/cpp/src/prims/transform_reduce_e_by_src_dst_key.cuh
@@ -95,6 +95,7 @@ template <bool edge_partition_src_key,
           typename EdgePartitionDstValueInputWrapper,
           typename EdgePartitionEdgeValueInputWrapper,
           typename EdgePartitionSrcDstKeyInputWrapper,
+          typename EdgePartitionEdgeMaskWrapper,
           typename EdgeOp,
           typename ValueIterator>
 __global__ static void transform_reduce_by_src_dst_key_hypersparse(
@@ -105,6 +106,9 @@ __global__ static void transform_reduce_by_src_dst_key_hypersparse(
   EdgePartitionDstValueInputWrapper edge_partition_dst_value_input,
   EdgePartitionEdgeValueInputWrapper edge_partition_e_value_input,
   EdgePartitionSrcDstKeyInputWrapper edge_partition_src_dst_key_input,
+  EdgePartitionEdgeMaskWrapper edge_partition_e_mask,
+  thrust::optional<raft::device_span<typename GraphViewType::edge_type const>>
+    edge_offsets_with_mask,
   EdgeOp e_op,
   typename GraphViewType::vertex_type* keys,
   ValueIterator value_iter)
@@ -129,19 +133,42 @@ __global__ static void transform_reduce_by_src_dst_key_hypersparse(
     edge_t local_degree{};
     thrust::tie(indices, edge_offset, local_degree) =
       edge_partition.local_edges(static_cast<vertex_t>(major_idx));
-    auto local_offset = edge_partition.local_offset(major_idx);
-    for (edge_t i = 0; i < local_degree; ++i) {
-      update_buffer_element<edge_partition_src_key, GraphViewType>(edge_partition,
-                                                                   major,
-                                                                   indices[i],
-                                                                   edge_offset + i,
-                                                                   edge_partition_src_value_input,
-                                                                   edge_partition_dst_value_input,
-                                                                   edge_partition_e_value_input,
-                                                                   edge_partition_src_dst_key_input,
-                                                                   e_op,
-                                                                   keys + local_offset + i,
-                                                                   value_iter + local_offset + i);
+    if (edge_partition_e_mask) {
+      auto major_offset          = edge_partition.major_offset_from_major_nocheck(major);
+      auto edge_offset_with_mask = (*edge_offsets_with_mask)[major_offset];
+      edge_t counter{0};
+      for (edge_t i = 0; i < local_degree; ++i) {
+        if ((*edge_partition_e_mask).get(edge_offset + i)) {
+          update_buffer_element<edge_partition_src_key, GraphViewType>(
+            edge_partition,
+            major,
+            indices[i],
+            edge_offset + i,
+            edge_partition_src_value_input,
+            edge_partition_dst_value_input,
+            edge_partition_e_value_input,
+            edge_partition_src_dst_key_input,
+            e_op,
+            keys + edge_offset_with_mask + counter,
+            value_iter + edge_offset_with_mask + counter);
+          ++counter;
+        }
+      }
+    } else {
+      for (edge_t i = 0; i < local_degree; ++i) {
+        update_buffer_element<edge_partition_src_key, GraphViewType>(
+          edge_partition,
+          major,
+          indices[i],
+          edge_offset + i,
+          edge_partition_src_value_input,
+          edge_partition_dst_value_input,
+          edge_partition_e_value_input,
+          edge_partition_src_dst_key_input,
+          e_op,
+          keys + edge_offset + i,
+          value_iter + edge_offset + i);
+      }
     }
 
     idx += gridDim.x * blockDim.x;
@@ -154,6 +181,7 @@ template <bool edge_partition_src_key,
           typename EdgePartitionDstValueInputWrapper,
           typename EdgePartitionEdgeValueInputWrapper,
           typename EdgePartitionSrcDstKeyInputWrapper,
+          typename EdgePartitionEdgeMaskWrapper,
           typename EdgeOp,
           typename ValueIterator>
 __global__ static void transform_reduce_by_src_dst_key_low_degree(
@@ -166,6 +194,9 @@ __global__ static void transform_reduce_by_src_dst_key_low_degree(
   EdgePartitionDstValueInputWrapper edge_partition_dst_value_input,
   EdgePartitionEdgeValueInputWrapper edge_partition_e_value_input,
   EdgePartitionSrcDstKeyInputWrapper edge_partition_src_dst_key_input,
+  EdgePartitionEdgeMaskWrapper edge_partition_e_mask,
+  thrust::optional<raft::device_span<typename GraphViewType::edge_type const>>
+    edge_offsets_with_mask,
   EdgeOp e_op,
   typename GraphViewType::vertex_type* keys,
   ValueIterator value_iter)
@@ -187,19 +218,41 @@ __global__ static void transform_reduce_by_src_dst_key_low_degree(
     edge_t local_degree{};
     thrust::tie(indices, edge_offset, local_degree) =
       edge_partition.local_edges(static_cast<vertex_t>(major_offset));
-    auto local_offset = edge_partition.local_offset(major_offset);
-    for (edge_t i = 0; i < local_degree; ++i) {
-      update_buffer_element<edge_partition_src_key, GraphViewType>(edge_partition,
-                                                                   major,
-                                                                   indices[i],
-                                                                   edge_offset + i,
-                                                                   edge_partition_src_value_input,
-                                                                   edge_partition_dst_value_input,
-                                                                   edge_partition_e_value_input,
-                                                                   edge_partition_src_dst_key_input,
-                                                                   e_op,
-                                                                   keys + local_offset + i,
-                                                                   value_iter + local_offset + i);
+    if (edge_partition_e_mask) {
+      auto edge_offset_with_mask = (*edge_offsets_with_mask)[major_offset];
+      edge_t counter{0};
+      for (edge_t i = 0; i < local_degree; ++i) {
+        if ((*edge_partition_e_mask).get(edge_offset + i)) {
+          update_buffer_element<edge_partition_src_key, GraphViewType>(
+            edge_partition,
+            major,
+            indices[i],
+            edge_offset + i,
+            edge_partition_src_value_input,
+            edge_partition_dst_value_input,
+            edge_partition_e_value_input,
+            edge_partition_src_dst_key_input,
+            e_op,
+            keys + edge_offset_with_mask + counter,
+            value_iter + edge_offset_with_mask + counter);
+          ++counter;
+        }
+      }
+    } else {
+      for (edge_t i = 0; i < local_degree; ++i) {
+        update_buffer_element<edge_partition_src_key, GraphViewType>(
+          edge_partition,
+          major,
+          indices[i],
+          edge_offset + i,
+          edge_partition_src_value_input,
+          edge_partition_dst_value_input,
+          edge_partition_e_value_input,
+          edge_partition_src_dst_key_input,
+          e_op,
+          keys + edge_offset + i,
+          value_iter + edge_offset + i);
+      }
     }
 
     idx += gridDim.x * blockDim.x;
@@ -212,6 +265,7 @@ template <bool edge_partition_src_key,
           typename EdgePartitionDstValueInputWrapper,
           typename EdgePartitionEdgeValueInputWrapper,
           typename EdgePartitionSrcDstKeyInputWrapper,
+          typename EdgePartitionEdgeMaskWrapper,
           typename EdgeOp,
           typename ValueIterator>
 __global__ static void transform_reduce_by_src_dst_key_mid_degree(
@@ -224,6 +278,9 @@ __global__ static void transform_reduce_by_src_dst_key_mid_degree(
   EdgePartitionDstValueInputWrapper edge_partition_dst_value_input,
   EdgePartitionEdgeValueInputWrapper edge_partition_e_value_input,
   EdgePartitionSrcDstKeyInputWrapper edge_partition_src_dst_key_input,
+  EdgePartitionEdgeMaskWrapper edge_partition_e_mask,
+  thrust::optional<raft::device_span<typename GraphViewType::edge_type const>>
+    edge_offsets_with_mask,
   EdgeOp e_op,
   typename GraphViewType::vertex_type* keys,
   ValueIterator value_iter)
@@ -238,6 +295,9 @@ __global__ static void transform_reduce_by_src_dst_key_mid_degree(
     static_cast<size_t>(major_range_first - edge_partition.major_range_first());
   size_t idx = static_cast<size_t>(tid / raft::warp_size());
 
+  using WarpScan = cub::WarpScan<edge_t, raft::warp_size()>;
+  __shared__ typename WarpScan::TempStorage temp_storage;
+
   while (idx < static_cast<size_t>(major_range_last - major_range_first)) {
     auto major_offset = major_start_offset + idx;
     auto major =
@@ -247,19 +307,49 @@ __global__ static void transform_reduce_by_src_dst_key_mid_degree(
     edge_t local_degree{};
     thrust::tie(indices, edge_offset, local_degree) =
       edge_partition.local_edges(static_cast<vertex_t>(major_offset));
-    auto local_offset = edge_partition.local_offset(major_offset);
-    for (edge_t i = lane_id; i < local_degree; i += raft::warp_size()) {
-      update_buffer_element<edge_partition_src_key, GraphViewType>(edge_partition,
-                                                                   major,
-                                                                   indices[i],
-                                                                   edge_offset + i,
-                                                                   edge_partition_src_value_input,
-                                                                   edge_partition_dst_value_input,
-                                                                   edge_partition_e_value_input,
-                                                                   edge_partition_src_dst_key_input,
-                                                                   e_op,
-                                                                   keys + local_offset + i,
-                                                                   value_iter + local_offset + i);
+    if (edge_partition_e_mask) {
+      // FIXME: it might be faster to update in warp-sync way
+      auto edge_offset_with_mask = (*edge_offsets_with_mask)[major_offset];
+      edge_t counter{0};
+      for (edge_t i = lane_id; i < local_degree; i += raft::warp_size()) {
+        if ((*edge_partition_e_mask).get(edge_offset + i)) { ++counter; }
+      }
+      edge_t offset_within_warp{};
+      WarpScan(temp_storage).ExclusiveSum(counter, offset_within_warp);
+      edge_offset_with_mask += offset_within_warp;
+      counter = 0;
+      for (edge_t i = lane_id; i < local_degree; i += raft::warp_size()) {
+        if ((*edge_partition_e_mask).get(edge_offset + i)) {
+          update_buffer_element<edge_partition_src_key, GraphViewType>(
+            edge_partition,
+            major,
+            indices[i],
+            edge_offset + i,
+            edge_partition_src_value_input,
+            edge_partition_dst_value_input,
+            edge_partition_e_value_input,
+            edge_partition_src_dst_key_input,
+            e_op,
+            keys + edge_offset_with_mask + counter,
+            value_iter + edge_offset_with_mask + counter);
+          ++counter;
+        }
+      }
+    } else {
+      for (edge_t i = lane_id; i < local_degree; i += raft::warp_size()) {
+        update_buffer_element<edge_partition_src_key, GraphViewType>(
+          edge_partition,
+          major,
+          indices[i],
+          edge_offset + i,
+          edge_partition_src_value_input,
+          edge_partition_dst_value_input,
+          edge_partition_e_value_input,
+          edge_partition_src_dst_key_input,
+          e_op,
+          keys + edge_offset + i,
+          value_iter + edge_offset + i);
+      }
     }
 
     idx += gridDim.x * (blockDim.x / raft::warp_size());
@@ -272,6 +362,7 @@ template <bool edge_partition_src_key,
           typename EdgePartitionDstValueInputWrapper,
           typename EdgePartitionEdgeValueInputWrapper,
           typename EdgePartitionSrcDstKeyInputWrapper,
+          typename EdgePartitionEdgeMaskWrapper,
           typename EdgeOp,
           typename ValueIterator>
 __global__ static void transform_reduce_by_src_dst_key_high_degree(
@@ -284,6 +375,9 @@ __global__ static void transform_reduce_by_src_dst_key_high_degree(
   EdgePartitionDstValueInputWrapper edge_partition_dst_value_input,
   EdgePartitionEdgeValueInputWrapper edge_partition_e_value_input,
   EdgePartitionSrcDstKeyInputWrapper edge_partition_src_dst_key_input,
+  EdgePartitionEdgeMaskWrapper edge_partition_e_mask,
+  thrust::optional<raft::device_span<typename GraphViewType::edge_type const>>
+    edge_offsets_with_mask,
   EdgeOp e_op,
   typename GraphViewType::vertex_type* keys,
   ValueIterator value_iter)
@@ -295,6 +389,9 @@ __global__ static void transform_reduce_by_src_dst_key_high_degree(
     static_cast<size_t>(major_range_first - edge_partition.major_range_first());
   auto idx = static_cast<size_t>(blockIdx.x);
 
+  using BlockScan = cub::BlockScan<edge_t, transform_reduce_e_by_src_dst_key_kernel_block_size>;
+  __shared__ typename BlockScan::TempStorage temp_storage;
+
   while (idx < static_cast<size_t>(major_range_last - major_range_first)) {
     auto major_offset = major_start_offset + idx;
     auto major =
@@ -304,19 +401,49 @@ __global__ static void transform_reduce_by_src_dst_key_high_degree(
     edge_t local_degree{};
     thrust::tie(indices, edge_offset, local_degree) =
       edge_partition.local_edges(static_cast<vertex_t>(major_offset));
-    auto local_offset = edge_partition.local_offset(major_offset);
-    for (edge_t i = threadIdx.x; i < local_degree; i += blockDim.x) {
-      update_buffer_element<edge_partition_src_key, GraphViewType>(edge_partition,
-                                                                   major,
-                                                                   indices[i],
-                                                                   edge_offset + i,
-                                                                   edge_partition_src_value_input,
-                                                                   edge_partition_dst_value_input,
-                                                                   edge_partition_e_value_input,
-                                                                   edge_partition_src_dst_key_input,
-                                                                   e_op,
-                                                                   keys + local_offset + i,
-                                                                   value_iter + local_offset + i);
+    if (edge_partition_e_mask) {
+      // FIXME: it might be faster to update in block-sync way
+      auto edge_offset_with_mask = (*edge_offsets_with_mask)[major_offset];
+      edge_t counter{0};
+      for (edge_t i = threadIdx.x; i < local_degree; i += blockDim.x) {
+        if ((*edge_partition_e_mask).get(edge_offset + i)) { ++counter; }
+      }
+      edge_t offset_within_block{};
+      BlockScan(temp_storage).ExclusiveSum(counter, offset_within_block);
+      edge_offset_with_mask += offset_within_block;
+      counter = 0;
+      for (edge_t i = threadIdx.x; i < local_degree; i += blockDim.x) {
+        if ((*edge_partition_e_mask).get(edge_offset + i)) {
+          update_buffer_element<edge_partition_src_key, GraphViewType>(
+            edge_partition,
+            major,
+            indices[i],
+            edge_offset + i,
+            edge_partition_src_value_input,
+            edge_partition_dst_value_input,
+            edge_partition_e_value_input,
+            edge_partition_src_dst_key_input,
+            e_op,
+            keys + edge_offset_with_mask + counter,
+            value_iter + edge_offset_with_mask + counter);
+          ++counter;
+        }
+      }
+    } else {
+      for (edge_t i = threadIdx.x; i < local_degree; i += blockDim.x) {
+        update_buffer_element<edge_partition_src_key, GraphViewType>(
+          edge_partition,
+          major,
+          indices[i],
+          edge_offset + i,
+          edge_partition_src_value_input,
+          edge_partition_dst_value_input,
+          edge_partition_e_value_input,
+          edge_partition_src_dst_key_input,
+          e_op,
+          keys + edge_offset + i,
+          value_iter + edge_offset + i);
+      }
     }
 
     idx += gridDim.x;
@@ -410,19 +537,41 @@ transform_reduce_e_by_src_dst_key(raft::handle_t const& handle,
       typename EdgeSrcDstKeyInputWrapper::value_iterator,
       typename EdgeSrcDstKeyInputWrapper::value_type>;
 
+  auto edge_mask_view = graph_view.edge_mask_view();
+
   rmm::device_uvector<vertex_t> keys(0, handle.get_stream());
   auto value_buffer = allocate_dataframe_buffer<T>(0, handle.get_stream());
   for (size_t i = 0; i < graph_view.number_of_local_edge_partitions(); ++i) {
     auto edge_partition =
       edge_partition_device_view_t<vertex_t, edge_t, GraphViewType::is_multi_gpu>(
         graph_view.local_edge_partition_view(i));
-
-    auto num_edges = edge_partition.number_of_edges();
-
-    rmm::device_uvector<vertex_t> tmp_keys(num_edges, handle.get_stream());
+    auto edge_partition_e_mask =
+      edge_mask_view
+        ? thrust::make_optional<
+            detail::edge_partition_edge_property_device_view_t<edge_t, uint32_t const*, bool>>(
+            *edge_mask_view, i)
+        : thrust::nullopt;
+
+    rmm::device_uvector<vertex_t> tmp_keys(0, handle.get_stream());
+    std::optional<rmm::device_uvector<edge_t>> edge_offsets_with_mask{std::nullopt};
+    if (edge_partition_e_mask) {
+      auto local_degrees = edge_partition.compute_local_degrees_with_mask(
+        (*edge_partition_e_mask).value_first(), handle.get_stream());
+      edge_offsets_with_mask =
+        rmm::device_uvector<edge_t>(edge_partition.major_range_size() + 1, handle.get_stream());
+      (*edge_offsets_with_mask).set_element_to_zero_async(0, handle.get_stream());
+      thrust::inclusive_scan(handle.get_thrust_policy(),
+                             local_degrees.begin(),
+                             local_degrees.end(),
+                             (*edge_offsets_with_mask).begin() + 1);
+      tmp_keys.resize((*edge_offsets_with_mask).back_element(handle.get_stream()),
+                      handle.get_stream());
+    } else {
+      tmp_keys.resize(edge_partition.number_of_edges(), handle.get_stream());
+    }
     auto tmp_value_buffer = allocate_dataframe_buffer<T>(tmp_keys.size(), handle.get_stream());
 
-    if (num_edges > 0) {
+    if (tmp_keys.size() > 0) {
       edge_partition_src_input_device_view_t edge_partition_src_value_input{};
       edge_partition_dst_input_device_view_t edge_partition_dst_value_input{};
       if constexpr (GraphViewType::is_storage_transposed) {
@@ -467,6 +616,11 @@ transform_reduce_e_by_src_dst_key(raft::handle_t const& handle,
               edge_partition_dst_value_input,
               edge_partition_e_value_input,
               edge_partition_src_dst_key_input,
+              edge_partition_e_mask,
+              edge_offsets_with_mask
+                ? thrust::make_optional<raft::device_span<edge_t const>>(
+                    (*edge_offsets_with_mask).data(), (*edge_offsets_with_mask).size())
+                : thrust::nullopt,
               e_op,
               tmp_keys.data(),
               get_dataframe_buffer_begin(tmp_value_buffer));
@@ -485,6 +639,11 @@ transform_reduce_e_by_src_dst_key(raft::handle_t const& handle,
               edge_partition_dst_value_input,
               edge_partition_e_value_input,
               edge_partition_src_dst_key_input,
+              edge_partition_e_mask,
+              edge_offsets_with_mask
+                ? thrust::make_optional<raft::device_span<edge_t const>>(
+                    (*edge_offsets_with_mask).data(), (*edge_offsets_with_mask).size())
+                : thrust::nullopt,
               e_op,
               tmp_keys.data(),
               get_dataframe_buffer_begin(tmp_value_buffer));
@@ -503,6 +662,11 @@ transform_reduce_e_by_src_dst_key(raft::handle_t const& handle,
               edge_partition_dst_value_input,
               edge_partition_e_value_input,
               edge_partition_src_dst_key_input,
+              edge_partition_e_mask,
+              edge_offsets_with_mask
+                ? thrust::make_optional<raft::device_span<edge_t const>>(
+                    (*edge_offsets_with_mask).data(), (*edge_offsets_with_mask).size())
+                : thrust::nullopt,
               e_op,
               tmp_keys.data(),
               get_dataframe_buffer_begin(tmp_value_buffer));
@@ -520,6 +684,11 @@ transform_reduce_e_by_src_dst_key(raft::handle_t const& handle,
               edge_partition_dst_value_input,
               edge_partition_e_value_input,
               edge_partition_src_dst_key_input,
+              edge_partition_e_mask,
+              edge_offsets_with_mask
+                ? thrust::make_optional<raft::device_span<edge_t const>>(
+                    (*edge_offsets_with_mask).data(), (*edge_offsets_with_mask).size())
+                : thrust::nullopt,
               e_op,
               tmp_keys.data(),
               get_dataframe_buffer_begin(tmp_value_buffer));
@@ -539,6 +708,11 @@ transform_reduce_e_by_src_dst_key(raft::handle_t const& handle,
             edge_partition_dst_value_input,
             edge_partition_e_value_input,
             edge_partition_src_dst_key_input,
+            edge_partition_e_mask,
+            edge_offsets_with_mask
+              ? thrust::make_optional<raft::device_span<edge_t const>>(
+                  (*edge_offsets_with_mask).data(), (*edge_offsets_with_mask).size())
+              : thrust::nullopt,
             e_op,
             tmp_keys.data(),
             get_dataframe_buffer_begin(tmp_value_buffer));
@@ -682,8 +856,6 @@ auto transform_reduce_e_by_src_key(raft::handle_t const& handle,
                              typename GraphViewType::vertex_type>::value);
   static_assert(ReduceOp::pure_function, "ReduceOp should be a pure function.");
 
-  CUGRAPH_EXPECTS(!graph_view.has_edge_mask(), "unimplemented.");
-
   if (do_expensive_check) {
     // currently, nothing to do
   }
@@ -772,8 +944,6 @@ auto transform_reduce_e_by_dst_key(raft::handle_t const& handle,
                              typename GraphViewType::vertex_type>::value);
   static_assert(ReduceOp::pure_function, "ReduceOp should be a pure function.");
 
-  CUGRAPH_EXPECTS(!graph_view.has_edge_mask(), "unimplemented.");
-
   if (do_expensive_check) {
     // currently, nothing to do
   }
diff --git a/cpp/tests/CMakeLists.txt b/cpp/tests/CMakeLists.txt
index c84711e1a69..4d37c93326d 100644
--- a/cpp/tests/CMakeLists.txt
+++ b/cpp/tests/CMakeLists.txt
@@ -396,12 +396,10 @@ ConfigureTest(RANDOM_WALKS_TEST sampling/sg_random_walks_test.cpp)
 ###################################################################################################
 # - NBR SAMPLING tests ----------------------------------------------------------------------------
 ConfigureTest(UNIFORM_NEIGHBOR_SAMPLING_TEST sampling/sg_uniform_neighbor_sampling.cu)
-target_link_libraries(UNIFORM_NEIGHBOR_SAMPLING_TEST PRIVATE cuco::cuco)
 
 ###################################################################################################
 # - SAMPLING_POST_PROCESSING tests ----------------------------------------------------------------
 ConfigureTest(SAMPLING_POST_PROCESSING_TEST sampling/sampling_post_processing_test.cu)
-target_link_libraries(SAMPLING_POST_PROCESSING_TEST PRIVATE cuco::cuco)
 
 ###################################################################################################
 # - Renumber tests --------------------------------------------------------------------------------
@@ -583,78 +581,79 @@ if(BUILD_CUGRAPH_MG_TESTS)
     ###############################################################################################
     # - MG PRIMS COUNT_IF_V tests -----------------------------------------------------------------
     ConfigureTestMG(MG_COUNT_IF_V_TEST prims/mg_count_if_v.cu)
-    target_link_libraries(MG_COUNT_IF_V_TEST PRIVATE cuco::cuco)
 
     ###############################################################################################
     # - MG PRIMS TRANSFORM_REDUCE_V_FRONTIER_OUTGOING_E_BY_DST tests ------------------------------
     ConfigureTestMG(MG_TRANSFORM_REDUCE_V_FRONTIER_OUTGOING_E_BY_DST_TEST
                     prims/mg_transform_reduce_v_frontier_outgoing_e_by_dst.cu)
-    target_link_libraries(MG_TRANSFORM_REDUCE_V_FRONTIER_OUTGOING_E_BY_DST_TEST PRIVATE cuco::cuco)
 
     ###############################################################################################
     # - MG PRIMS REDUCE_V tests -------------------------------------------------------------------
     ConfigureTestMG(MG_REDUCE_V_TEST prims/mg_reduce_v.cu)
-    target_link_libraries(MG_REDUCE_V_TEST PRIVATE cuco::cuco)
 
     ###############################################################################################
     # - MG PRIMS TRANSFORM_REDUCE_V tests ---------------------------------------------------------
     ConfigureTestMG(MG_TRANSFORM_REDUCE_V_TEST prims/mg_transform_reduce_v.cu)
-    target_link_libraries(MG_TRANSFORM_REDUCE_V_TEST PRIVATE cuco::cuco)
 
     ###############################################################################################
     # - MG PRIMS TRANSFORM_REDUCE_E tests ---------------------------------------------------------
     ConfigureTestMG(MG_TRANSFORM_REDUCE_E_TEST prims/mg_transform_reduce_e.cu)
-    target_link_libraries(MG_TRANSFORM_REDUCE_E_TEST PRIVATE cuco::cuco)
+
+    ###############################################################################################
+    # - MG PRIMS TRANSFORM_REDUCE_E _BY_SRC_DST_KEY tests -----------------------------------------
+    ConfigureTestMG(MG_TRANSFORM_REDUCE_E_BY_SRC_DST_KEY_TEST
+                    prims/mg_transform_reduce_e_by_src_dst_key.cu)
 
     ###############################################################################################
     # - MG PRIMS TRANSFORM_E tests ----------------------------------------------------------------
     ConfigureTestMG(MG_TRANSFORM_E_TEST prims/mg_transform_e.cu)
-    target_link_libraries(MG_TRANSFORM_E_TEST PRIVATE cuco::cuco)
 
     ###############################################################################################
     # - MG PRIMS COUNT_IF_E tests -----------------------------------------------------------------
     ConfigureTestMG(MG_COUNT_IF_E_TEST prims/mg_count_if_e.cu)
-    target_link_libraries(MG_COUNT_IF_E_TEST PRIVATE cuco::cuco)
 
     ###############################################################################################
     # - MG PRIMS PER_V_TRANSFORM_REDUCE_INCOMING_OUTGOING_E tests ---------------------------------
     ConfigureTestMG(MG_PER_V_TRANSFORM_REDUCE_INCOMING_OUTGOING_E_TEST
       prims/mg_per_v_transform_reduce_incoming_outgoing_e.cu)
-    target_link_libraries(MG_PER_V_TRANSFORM_REDUCE_INCOMING_OUTGOING_E_TEST PRIVATE cuco::cuco)
+
+    ###############################################################################################
+    # - MG PRIMS PER_V_TRANSFORM_REDUCE_DST_KEY_AGGREGATED_OUTGOING_E tests -----------------------
+    ConfigureTestMG(MG_PER_V_TRANSFORM_REDUCE_DST_KEY_AGGREGATED_OUTGOING_E_TEST
+      prims/mg_per_v_transform_reduce_dst_key_aggregated_outgoing_e.cu)
 
     ###############################################################################################
     # - MG PRIMS EXTRACT_TRANSFORM_E tests --------------------------------------------------------
     ConfigureTestMG(MG_EXTRACT_TRANSFORM_E_TEST prims/mg_extract_transform_e.cu)
-    target_link_libraries(MG_EXTRACT_TRANSFORM_E_TEST PRIVATE cuco::cuco)
 
     ###############################################################################################
     # - MG PRIMS EXTRACT_TRANSFORM_V_FRONTIER_OUTGOING_E tests ------------------------------------
     ConfigureTestMG(MG_EXTRACT_TRANSFORM_V_FRONTIER_OUTGOING_E_TEST
                     prims/mg_extract_transform_v_frontier_outgoing_e.cu)
-    target_link_libraries(MG_EXTRACT_TRANSFORM_V_FRONTIER_OUTGOING_E_TEST PRIVATE cuco::cuco)
 
     ###############################################################################################
     # - MG PRIMS PER_V_RANDOM_SELECT_TRANSFORM_OUTGOING_E tests -----------------------------------
     ConfigureTestMG(MG_PER_V_RANDOM_SELECT_TRANSFORM_OUTGOING_E_TEST
                     prims/mg_per_v_random_select_transform_outgoing_e.cu)
-    target_link_libraries(MG_PER_V_RANDOM_SELECT_TRANSFORM_OUTGOING_E_TEST PRIVATE cuco::cuco)
 
     ###############################################################################################
     # - MG PRIMS PER_V_PAIR_TRANSFORM_DST_NBR_INTERSECTION tests ----------------------------------
     ConfigureTestMG(MG_PER_V_PAIR_TRANSFORM_DST_NBR_INTERSECTION_TEST
                     prims/mg_per_v_pair_transform_dst_nbr_intersection.cu)
-    target_link_libraries(MG_PER_V_PAIR_TRANSFORM_DST_NBR_INTERSECTION_TEST PRIVATE cuco::cuco)
+
+    ###############################################################################################
+    # - MG PRIMS TRANSFORM_REDUCE_DST_NBR_INTERSECTION OF_E_ENDPOINTS_BY_V tests ------------------
+    ConfigureTestMG(MG_TRANSFORM_REDUCE_DST_NBR_INTERSECTION_BY_E_ENDPOINTS_BY_V_TEST
+                    prims/mg_transform_reduce_dst_nbr_intersection_of_e_endpoints_by_v.cu)
 
     ###############################################################################################
     # - MG PRIMS PER_V_PAIR_TRANSFORM_DST_NBR_WEIGHTED_INTERSECTION tests -------------------------
     ConfigureTestMG(MG_PER_V_PAIR_TRANSFORM_DST_NBR_WEIGHTED_INTERSECTION_TEST
-    prims/mg_per_v_pair_transform_dst_nbr_weighted_intersection.cu)
-    target_link_libraries(MG_PER_V_PAIR_TRANSFORM_DST_NBR_WEIGHTED_INTERSECTION_TEST PRIVATE cuco::cuco)
+                    prims/mg_per_v_pair_transform_dst_nbr_weighted_intersection.cu)
 
     ###############################################################################################
     # - MG NBR SAMPLING tests ---------------------------------------------------------------------
     ConfigureTestMG(MG_UNIFORM_NEIGHBOR_SAMPLING_TEST sampling/mg_uniform_neighbor_sampling.cu)
-    target_link_libraries(MG_UNIFORM_NEIGHBOR_SAMPLING_TEST PRIVATE cuco::cuco)
 
     ###############################################################################################
     # - MG RANDOM_WALKS tests ---------------------------------------------------------------------
diff --git a/cpp/tests/prims/mg_per_v_transform_reduce_dst_key_aggregated_outgoing_e.cu b/cpp/tests/prims/mg_per_v_transform_reduce_dst_key_aggregated_outgoing_e.cu
new file mode 100644
index 00000000000..af56807746a
--- /dev/null
+++ b/cpp/tests/prims/mg_per_v_transform_reduce_dst_key_aggregated_outgoing_e.cu
@@ -0,0 +1,599 @@
+/*
+ * Copyright (c) 2021-2024, NVIDIA CORPORATION.
+ *
+ * Licensed 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.
+ */
+
+#include "prims/per_v_transform_reduce_dst_key_aggregated_outgoing_e.cuh"
+#include "prims/reduce_op.cuh"
+#include "prims/update_edge_src_dst_property.cuh"
+#include "property_generator.cuh"
+#include "result_compare.cuh"
+#include "utilities/base_fixture.hpp"
+#include "utilities/device_comm_wrapper.hpp"
+#include "utilities/mg_utilities.hpp"
+#include "utilities/test_graphs.hpp"
+#include "utilities/test_utilities.hpp"
+#include "utilities/thrust_wrapper.hpp"
+
+#include <cugraph/algorithms.hpp>
+#include <cugraph/edge_partition_view.hpp>
+#include <cugraph/edge_src_dst_property.hpp>
+#include <cugraph/graph_view.hpp>
+#include <cugraph/utilities/dataframe_buffer.hpp>
+#include <cugraph/utilities/high_res_timer.hpp>
+#include <cugraph/utilities/thrust_tuple_utils.hpp>
+
+#include <raft/comms/mpi_comms.hpp>
+#include <raft/core/comms.hpp>
+#include <raft/core/handle.hpp>
+
+#include <rmm/device_scalar.hpp>
+#include <rmm/device_uvector.hpp>
+
+#include <thrust/count.h>
+#include <thrust/distance.h>
+#include <thrust/equal.h>
+#include <thrust/functional.h>
+#include <thrust/iterator/counting_iterator.h>
+#include <thrust/optional.h>
+#include <thrust/transform.h>
+#include <thrust/tuple.h>
+
+#include <cuco/hash_functions.cuh>
+
+#include <gtest/gtest.h>
+
+#include <random>
+#include <sstream>
+
+template <typename vertex_t, typename edge_value_t, typename result_t>
+struct key_aggregated_e_op_t {
+  __device__ result_t operator()(vertex_t src,
+                                 vertex_t key,
+                                 result_t src_property,
+                                 result_t key_property,
+                                 edge_value_t edge_property) const
+  {
+    if (src_property < key_property) {
+      return src_property;
+    } else {
+      return key_property;
+    }
+  }
+};
+
+struct Prims_Usecase {
+  bool test_weighted{false};
+  bool edge_masking{false};
+  bool check_correctness{true};
+};
+
+template <typename input_usecase_t>
+class Tests_MGPerVTransformReduceDstKeyAggregatedOutgoingE
+  : public ::testing::TestWithParam<std::tuple<Prims_Usecase, input_usecase_t>> {
+ public:
+  Tests_MGPerVTransformReduceDstKeyAggregatedOutgoingE() {}
+
+  static void SetUpTestCase() { handle_ = cugraph::test::initialize_mg_handle(); }
+
+  static void TearDownTestCase() { handle_.reset(); }
+
+  virtual void SetUp() {}
+  virtual void TearDown() {}
+
+  // Compare the results of per_v_transform_reduce_incoming|outgoing_e primitive
+  template <typename vertex_t, typename edge_t, typename weight_t, typename result_t>
+  void run_current_test(Prims_Usecase const& prims_usecase, input_usecase_t const& input_usecase)
+  {
+    HighResTimer hr_timer{};
+
+    auto const comm_rank = handle_->get_comms().get_rank();
+
+    // 1. create MG graph
+
+    if (cugraph::test::g_perf) {
+      RAFT_CUDA_TRY(cudaDeviceSynchronize());  // for consistent performance measurement
+      handle_->get_comms().barrier();
+      hr_timer.start("MG Construct graph");
+    }
+
+    auto [mg_graph, mg_edge_weights, mg_renumber_map] =
+      cugraph::test::construct_graph<vertex_t, edge_t, weight_t, false, true>(
+        *handle_, input_usecase, prims_usecase.test_weighted, true);
+
+    if (cugraph::test::g_perf) {
+      RAFT_CUDA_TRY(cudaDeviceSynchronize());  // for consistent performance measurement
+      handle_->get_comms().barrier();
+      hr_timer.stop();
+      hr_timer.display_and_clear(std::cout);
+    }
+
+    auto mg_graph_view = mg_graph.view();
+    auto mg_edge_weight_view =
+      mg_edge_weights ? std::make_optional((*mg_edge_weights).view()) : std::nullopt;
+
+    std::optional<cugraph::edge_property_t<decltype(mg_graph_view), bool>> edge_mask{std::nullopt};
+    if (prims_usecase.edge_masking) {
+      edge_mask =
+        cugraph::test::generate<vertex_t, bool>::edge_property(*handle_, mg_graph_view, 2);
+      mg_graph_view.attach_edge_mask((*edge_mask).view());
+    }
+
+    // 2. run MG per_v_transform_reduce_dst_key_aggregated_outgoing_e
+
+    const int vertex_prop_hash_bin_count = 5;
+    const int key_hash_bin_count         = 10;
+    const int key_prop_hash_bin_count    = 20;
+    const int initial_value              = 4;
+
+    auto property_initial_value =
+      cugraph::test::generate<vertex_t, result_t>::initial_value(initial_value);
+
+    auto mg_vertex_prop = cugraph::test::generate<vertex_t, result_t>::vertex_property(
+      *handle_, *mg_renumber_map, vertex_prop_hash_bin_count);
+    auto mg_src_prop = cugraph::test::generate<vertex_t, result_t>::src_property(
+      *handle_, mg_graph_view, mg_vertex_prop);
+
+    auto mg_vertex_key = cugraph::test::generate<vertex_t, vertex_t>::vertex_property(
+      *handle_, *mg_renumber_map, key_hash_bin_count);
+    auto mg_dst_key = cugraph::test::generate<vertex_t, vertex_t>::dst_property(
+      *handle_, mg_graph_view, mg_vertex_key);
+
+    rmm::device_uvector<vertex_t> mg_kv_store_keys(comm_rank == 0 ? key_hash_bin_count : int{0},
+                                                   handle_->get_stream());
+    thrust::sequence(
+      handle_->get_thrust_policy(), mg_kv_store_keys.begin(), mg_kv_store_keys.end(), vertex_t{0});
+    mg_kv_store_keys = cugraph::detail::shuffle_ext_vertices_to_local_gpu_by_vertex_partitioning(
+      *handle_, std::move(mg_kv_store_keys));
+    auto mg_kv_store_values = cugraph::test::generate<vertex_t, result_t>::vertex_property(
+      *handle_, mg_kv_store_keys, key_prop_hash_bin_count);
+
+    static_assert(std::is_same_v<result_t, int> ||
+                  std::is_same_v<result_t, thrust::tuple<int, float>>);
+    result_t invalid_value{};
+    if constexpr (std::is_same_v<result_t, int>) {
+      invalid_value = std::numeric_limits<int>::max();
+    } else {
+      invalid_value =
+        thrust::make_tuple(std::numeric_limits<int>::max(), std::numeric_limits<float>::max());
+    }
+    cugraph::kv_store_t<vertex_t, result_t, false> mg_kv_store(
+      mg_kv_store_keys.begin(),
+      mg_kv_store_keys.end(),
+      cugraph::get_dataframe_buffer_begin(mg_kv_store_values),
+      cugraph::invalid_vertex_id<vertex_t>::value,
+      invalid_value,
+      handle_->get_stream());
+
+    enum class reduction_type_t { PLUS, ELEMWISE_MIN, ELEMWISE_MAX };
+    std::array<reduction_type_t, 3> reduction_types = {
+      reduction_type_t::PLUS, reduction_type_t::ELEMWISE_MIN, reduction_type_t::ELEMWISE_MAX};
+
+    std::vector<decltype(cugraph::allocate_dataframe_buffer<result_t>(0, rmm::cuda_stream_view{}))>
+      mg_results{};
+    mg_results.reserve(reduction_types.size());
+
+    for (size_t i = 0; i < reduction_types.size(); ++i) {
+      mg_results.push_back(cugraph::allocate_dataframe_buffer<result_t>(
+        mg_graph_view.local_vertex_partition_range_size(), handle_->get_stream()));
+
+      if (cugraph::test::g_perf) {
+        RAFT_CUDA_TRY(cudaDeviceSynchronize());  // for consistent performance measurement
+        handle_->get_comms().barrier();
+        hr_timer.start("MG per_v_transform_reduce_outgoing_e");
+      }
+
+      switch (reduction_types[i]) {
+        case reduction_type_t::PLUS:
+          if (mg_edge_weight_view) {
+            per_v_transform_reduce_dst_key_aggregated_outgoing_e(
+              *handle_,
+              mg_graph_view,
+              mg_src_prop.view(),
+              *mg_edge_weight_view,
+              mg_dst_key.view(),
+              mg_kv_store.view(),
+              key_aggregated_e_op_t<vertex_t, weight_t, result_t>{},
+              property_initial_value,
+              cugraph::reduce_op::plus<result_t>{},
+              cugraph::get_dataframe_buffer_begin(mg_results[i]));
+          } else {
+            per_v_transform_reduce_dst_key_aggregated_outgoing_e(
+              *handle_,
+              mg_graph_view,
+              mg_src_prop.view(),
+              cugraph::edge_dummy_property_t{}.view(),
+              mg_dst_key.view(),
+              mg_kv_store.view(),
+              key_aggregated_e_op_t<vertex_t, thrust::nullopt_t, result_t>{},
+              property_initial_value,
+              cugraph::reduce_op::plus<result_t>{},
+              cugraph::get_dataframe_buffer_begin(mg_results[i]));
+          }
+          break;
+        case reduction_type_t::ELEMWISE_MIN:
+          if (mg_edge_weight_view) {
+            per_v_transform_reduce_dst_key_aggregated_outgoing_e(
+              *handle_,
+              mg_graph_view,
+              mg_src_prop.view(),
+              *mg_edge_weight_view,
+              mg_dst_key.view(),
+              mg_kv_store.view(),
+              key_aggregated_e_op_t<vertex_t, weight_t, result_t>{},
+              property_initial_value,
+              cugraph::reduce_op::elementwise_minimum<result_t>{},
+              cugraph::get_dataframe_buffer_begin(mg_results[i]));
+          } else {
+            per_v_transform_reduce_dst_key_aggregated_outgoing_e(
+              *handle_,
+              mg_graph_view,
+              mg_src_prop.view(),
+              cugraph::edge_dummy_property_t{}.view(),
+              mg_dst_key.view(),
+              mg_kv_store.view(),
+              key_aggregated_e_op_t<vertex_t, thrust::nullopt_t, result_t>{},
+              property_initial_value,
+              cugraph::reduce_op::elementwise_minimum<result_t>{},
+              cugraph::get_dataframe_buffer_begin(mg_results[i]));
+          }
+          break;
+        case reduction_type_t::ELEMWISE_MAX:
+          if (mg_edge_weight_view) {
+            per_v_transform_reduce_dst_key_aggregated_outgoing_e(
+              *handle_,
+              mg_graph_view,
+              mg_src_prop.view(),
+              *mg_edge_weight_view,
+              mg_dst_key.view(),
+              mg_kv_store.view(),
+              key_aggregated_e_op_t<vertex_t, weight_t, result_t>{},
+              property_initial_value,
+              cugraph::reduce_op::elementwise_maximum<result_t>{},
+              cugraph::get_dataframe_buffer_begin(mg_results[i]));
+          } else {
+            per_v_transform_reduce_dst_key_aggregated_outgoing_e(
+              *handle_,
+              mg_graph_view,
+              mg_src_prop.view(),
+              cugraph::edge_dummy_property_t{}.view(),
+              mg_dst_key.view(),
+              mg_kv_store.view(),
+              key_aggregated_e_op_t<vertex_t, thrust::nullopt_t, result_t>{},
+              property_initial_value,
+              cugraph::reduce_op::elementwise_maximum<result_t>{},
+              cugraph::get_dataframe_buffer_begin(mg_results[i]));
+          }
+          break;
+        default: FAIL() << "should not be reached.";
+      }
+
+      if (cugraph::test::g_perf) {
+        RAFT_CUDA_TRY(cudaDeviceSynchronize());  // for consistent performance measurement
+        handle_->get_comms().barrier();
+        hr_timer.stop();
+        hr_timer.display_and_clear(std::cout);
+      }
+    }
+
+    // 3. compare SG & MG results
+
+    if (prims_usecase.check_correctness) {
+      cugraph::graph_t<vertex_t, edge_t, false, false> sg_graph(*handle_);
+      std::optional<
+        cugraph::edge_property_t<cugraph::graph_view_t<vertex_t, edge_t, false, false>, weight_t>>
+        sg_edge_weights{std::nullopt};
+      std::tie(sg_graph, sg_edge_weights, std::ignore) = cugraph::test::mg_graph_to_sg_graph(
+        *handle_,
+        mg_graph_view,
+        std::optional<cugraph::edge_property_view_t<edge_t, weight_t const*>>{std::nullopt},
+        std::make_optional<raft::device_span<vertex_t const>>((*mg_renumber_map).data(),
+                                                              (*mg_renumber_map).size()),
+        false);
+
+      for (size_t i = 0; i < reduction_types.size(); ++i) {
+        auto mg_aggregate_results =
+          cugraph::allocate_dataframe_buffer<result_t>(0, handle_->get_stream());
+
+        static_assert(cugraph::is_arithmetic_or_thrust_tuple_of_arithmetic<result_t>::value);
+        if constexpr (std::is_arithmetic_v<result_t>) {
+          std::tie(std::ignore, mg_aggregate_results) =
+            cugraph::test::mg_vertex_property_values_to_sg_vertex_property_values(
+              *handle_,
+              std::make_optional<raft::device_span<vertex_t const>>((*mg_renumber_map).data(),
+                                                                    (*mg_renumber_map).size()),
+              mg_graph_view.local_vertex_partition_range(),
+              std::optional<raft::device_span<vertex_t const>>{std::nullopt},
+              std::optional<raft::device_span<vertex_t const>>{std::nullopt},
+              raft::device_span<result_t const>(mg_results[i].data(), mg_results[i].size()));
+        } else {
+          std::tie(std::ignore, std::get<0>(mg_aggregate_results)) =
+            cugraph::test::mg_vertex_property_values_to_sg_vertex_property_values(
+              *handle_,
+              std::make_optional<raft::device_span<vertex_t const>>((*mg_renumber_map).data(),
+                                                                    (*mg_renumber_map).size()),
+              mg_graph_view.local_vertex_partition_range(),
+              std::optional<raft::device_span<vertex_t const>>{std::nullopt},
+              std::optional<raft::device_span<vertex_t const>>{std::nullopt},
+              raft::device_span<typename thrust::tuple_element<0, result_t>::type const>(
+                std::get<0>(mg_results[i]).data(), std::get<0>(mg_results[i]).size()));
+
+          std::tie(std::ignore, std::get<1>(mg_aggregate_results)) =
+            cugraph::test::mg_vertex_property_values_to_sg_vertex_property_values(
+              *handle_,
+              std::make_optional<raft::device_span<vertex_t const>>((*mg_renumber_map).data(),
+                                                                    (*mg_renumber_map).size()),
+              mg_graph_view.local_vertex_partition_range(),
+              std::optional<raft::device_span<vertex_t const>>{std::nullopt},
+              std::optional<raft::device_span<vertex_t const>>{std::nullopt},
+              raft::device_span<typename thrust::tuple_element<1, result_t>::type const>(
+                std::get<1>(mg_results[i]).data(), std::get<1>(mg_results[i]).size()));
+        }
+
+        if (handle_->get_comms().get_rank() == int{0}) {
+          auto sg_graph_view = sg_graph.view();
+          auto sg_edge_weight_view =
+            sg_edge_weights ? std::make_optional((*sg_edge_weights).view()) : std::nullopt;
+
+          auto sg_vertex_prop = cugraph::test::generate<vertex_t, result_t>::vertex_property(
+            *handle_,
+            thrust::make_counting_iterator(sg_graph_view.local_vertex_partition_range_first()),
+            thrust::make_counting_iterator(sg_graph_view.local_vertex_partition_range_last()),
+            vertex_prop_hash_bin_count);
+          auto sg_src_prop = cugraph::test::generate<vertex_t, result_t>::src_property(
+            *handle_, sg_graph_view, sg_vertex_prop);
+
+          auto sg_vertex_key = cugraph::test::generate<vertex_t, vertex_t>::vertex_property(
+            *handle_,
+            thrust::make_counting_iterator(sg_graph_view.local_vertex_partition_range_first()),
+            thrust::make_counting_iterator(sg_graph_view.local_vertex_partition_range_last()),
+            key_hash_bin_count);
+          auto sg_dst_key = cugraph::test::generate<vertex_t, vertex_t>::dst_property(
+            *handle_, sg_graph_view, sg_vertex_key);
+
+          rmm::device_uvector<vertex_t> sg_kv_store_keys(key_hash_bin_count, handle_->get_stream());
+          thrust::sequence(handle_->get_thrust_policy(),
+                           sg_kv_store_keys.begin(),
+                           sg_kv_store_keys.end(),
+                           vertex_t{0});
+          auto sg_kv_store_values = cugraph::test::generate<vertex_t, result_t>::vertex_property(
+            *handle_, sg_kv_store_keys, key_prop_hash_bin_count);
+
+          cugraph::kv_store_t<vertex_t, result_t, false> sg_kv_store(
+            sg_kv_store_keys.begin(),
+            sg_kv_store_keys.end(),
+            cugraph::get_dataframe_buffer_begin(sg_kv_store_values),
+            cugraph::invalid_vertex_id<vertex_t>::value,
+            invalid_value,
+            handle_->get_stream());
+
+          cugraph::test::vector_result_compare compare{*handle_};
+
+          auto global_result = cugraph::allocate_dataframe_buffer<result_t>(
+            sg_graph_view.local_vertex_partition_range_size(), handle_->get_stream());
+
+          switch (reduction_types[i]) {
+            case reduction_type_t::PLUS:
+              if (sg_edge_weight_view) {
+                per_v_transform_reduce_dst_key_aggregated_outgoing_e(
+                  *handle_,
+                  sg_graph_view,
+                  sg_src_prop.view(),
+                  *sg_edge_weight_view,
+                  sg_dst_key.view(),
+                  sg_kv_store.view(),
+                  key_aggregated_e_op_t<vertex_t, weight_t, result_t>{},
+                  property_initial_value,
+                  cugraph::reduce_op::plus<result_t>{},
+                  cugraph::get_dataframe_buffer_begin(global_result));
+              } else {
+                per_v_transform_reduce_dst_key_aggregated_outgoing_e(
+                  *handle_,
+                  sg_graph_view,
+                  sg_src_prop.view(),
+                  cugraph::edge_dummy_property_t{}.view(),
+                  sg_dst_key.view(),
+                  sg_kv_store.view(),
+                  key_aggregated_e_op_t<vertex_t, thrust::nullopt_t, result_t>{},
+                  property_initial_value,
+                  cugraph::reduce_op::plus<result_t>{},
+                  cugraph::get_dataframe_buffer_begin(global_result));
+              }
+              break;
+            case reduction_type_t::ELEMWISE_MIN:
+              if (sg_edge_weight_view) {
+                per_v_transform_reduce_dst_key_aggregated_outgoing_e(
+                  *handle_,
+                  sg_graph_view,
+                  sg_src_prop.view(),
+                  *sg_edge_weight_view,
+                  sg_dst_key.view(),
+                  sg_kv_store.view(),
+                  key_aggregated_e_op_t<vertex_t, weight_t, result_t>{},
+                  property_initial_value,
+                  cugraph::reduce_op::elementwise_minimum<result_t>{},
+                  cugraph::get_dataframe_buffer_begin(global_result));
+              } else {
+                per_v_transform_reduce_dst_key_aggregated_outgoing_e(
+                  *handle_,
+                  sg_graph_view,
+                  sg_src_prop.view(),
+                  cugraph::edge_dummy_property_t{}.view(),
+                  sg_dst_key.view(),
+                  sg_kv_store.view(),
+                  key_aggregated_e_op_t<vertex_t, thrust::nullopt_t, result_t>{},
+                  property_initial_value,
+                  cugraph::reduce_op::elementwise_minimum<result_t>{},
+                  cugraph::get_dataframe_buffer_begin(global_result));
+              }
+              break;
+            case reduction_type_t::ELEMWISE_MAX:
+              if (sg_edge_weight_view) {
+                per_v_transform_reduce_dst_key_aggregated_outgoing_e(
+                  *handle_,
+                  sg_graph_view,
+                  sg_src_prop.view(),
+                  *sg_edge_weight_view,
+                  sg_dst_key.view(),
+                  sg_kv_store.view(),
+                  key_aggregated_e_op_t<vertex_t, weight_t, result_t>{},
+                  property_initial_value,
+                  cugraph::reduce_op::elementwise_maximum<result_t>{},
+                  cugraph::get_dataframe_buffer_begin(global_result));
+              } else {
+                per_v_transform_reduce_dst_key_aggregated_outgoing_e(
+                  *handle_,
+                  sg_graph_view,
+                  sg_src_prop.view(),
+                  cugraph::edge_dummy_property_t{}.view(),
+                  sg_dst_key.view(),
+                  sg_kv_store.view(),
+                  key_aggregated_e_op_t<vertex_t, thrust::nullopt_t, result_t>{},
+                  property_initial_value,
+                  cugraph::reduce_op::elementwise_maximum<result_t>{},
+                  cugraph::get_dataframe_buffer_begin(global_result));
+              }
+              break;
+            default: FAIL() << "should not be reached.";
+          }
+
+          ASSERT_TRUE(compare(mg_aggregate_results, global_result));
+        }
+      }
+    }
+  }
+
+ private:
+  static std::unique_ptr<raft::handle_t> handle_;
+};
+
+template <typename input_usecase_t>
+std::unique_ptr<raft::handle_t>
+  Tests_MGPerVTransformReduceDstKeyAggregatedOutgoingE<input_usecase_t>::handle_ = nullptr;
+
+using Tests_MGPerVTransformReduceDstKeyAggregatedOutgoingE_File =
+  Tests_MGPerVTransformReduceDstKeyAggregatedOutgoingE<cugraph::test::File_Usecase>;
+using Tests_MGPerVTransformReduceDstKeyAggregatedOutgoingE_Rmat =
+  Tests_MGPerVTransformReduceDstKeyAggregatedOutgoingE<cugraph::test::Rmat_Usecase>;
+
+// FIXME: this tests do not build as cugrpah::kv_store_t has a build error when use_binary_search =
+// false and value_t is thrust::tuple, this will be fixed in a separate PR
+#if 0
+TEST_P(Tests_MGPerVTransformReduceDstKeyAggregatedOutgoingE_File,
+       CheckInt32Int32FloatTupleIntFloatTransposeFalse)
+{
+  auto param = GetParam();
+  run_current_test<int32_t, int32_t, float, thrust::tuple<int, float>>(std::get<0>(param),
+                                                                              std::get<1>(param));
+}
+
+TEST_P(Tests_MGPerVTransformReduceDstKeyAggregatedOutgoingE_Rmat,
+       CheckInt32Int32FloatTupleIntFloatTransposeFalse)
+{
+  auto param = GetParam();
+  run_current_test<int32_t, int32_t, float, thrust::tuple<int, float>>(
+    std::get<0>(param),
+    cugraph::test::override_Rmat_Usecase_with_cmd_line_arguments(std::get<1>(param)));
+}
+
+TEST_P(Tests_MGPerVTransformReduceDstKeyAggregatedOutgoingE_Rmat,
+       CheckInt32Int64FloatTupleIntFloatTransposeFalse)
+{
+  auto param = GetParam();
+  run_current_test<int32_t, int64_t, float, thrust::tuple<int, float>>(
+    std::get<0>(param),
+    cugraph::test::override_Rmat_Usecase_with_cmd_line_arguments(std::get<1>(param)));
+}
+
+TEST_P(Tests_MGPerVTransformReduceDstKeyAggregatedOutgoingE_Rmat,
+       CheckInt64Int64FloatTupleIntFloatTransposeFalse)
+{
+  auto param = GetParam();
+  run_current_test<int64_t, int64_t, float, thrust::tuple<int, float>>(
+    std::get<0>(param),
+    cugraph::test::override_Rmat_Usecase_with_cmd_line_arguments(std::get<1>(param)));
+}
+#endif
+
+TEST_P(Tests_MGPerVTransformReduceDstKeyAggregatedOutgoingE_File,
+       CheckInt32Int32FloatTransposeFalse)
+{
+  auto param = GetParam();
+  run_current_test<int32_t, int32_t, float, int>(std::get<0>(param), std::get<1>(param));
+}
+
+TEST_P(Tests_MGPerVTransformReduceDstKeyAggregatedOutgoingE_Rmat,
+       CheckInt32Int32FloatTransposeFalse)
+{
+  auto param = GetParam();
+  run_current_test<int32_t, int32_t, float, int>(
+    std::get<0>(param),
+    cugraph::test::override_Rmat_Usecase_with_cmd_line_arguments(std::get<1>(param)));
+}
+
+TEST_P(Tests_MGPerVTransformReduceDstKeyAggregatedOutgoingE_Rmat,
+       CheckInt32Int64FloatTransposeFalse)
+{
+  auto param = GetParam();
+  run_current_test<int32_t, int64_t, float, int>(
+    std::get<0>(param),
+    cugraph::test::override_Rmat_Usecase_with_cmd_line_arguments(std::get<1>(param)));
+}
+
+TEST_P(Tests_MGPerVTransformReduceDstKeyAggregatedOutgoingE_Rmat,
+       CheckInt64Int64FloatTransposeFalse)
+{
+  auto param = GetParam();
+  run_current_test<int64_t, int64_t, float, int>(
+    std::get<0>(param),
+    cugraph::test::override_Rmat_Usecase_with_cmd_line_arguments(std::get<1>(param)));
+}
+
+INSTANTIATE_TEST_SUITE_P(
+  file_test,
+  Tests_MGPerVTransformReduceDstKeyAggregatedOutgoingE_File,
+  ::testing::Combine(
+    ::testing::Values(Prims_Usecase{false, false, true},
+                      Prims_Usecase{false, true, true},
+                      Prims_Usecase{true, false, true},
+                      Prims_Usecase{true, true, true}),
+    ::testing::Values(cugraph::test::File_Usecase("test/datasets/karate.mtx"),
+                      cugraph::test::File_Usecase("test/datasets/web-Google.mtx"),
+                      cugraph::test::File_Usecase("test/datasets/ljournal-2008.mtx"),
+                      cugraph::test::File_Usecase("test/datasets/webbase-1M.mtx"))));
+
+INSTANTIATE_TEST_SUITE_P(rmat_small_test,
+                         Tests_MGPerVTransformReduceDstKeyAggregatedOutgoingE_Rmat,
+                         ::testing::Combine(::testing::Values(Prims_Usecase{false, false, true},
+                                                              Prims_Usecase{false, true, true},
+                                                              Prims_Usecase{true, false, true},
+                                                              Prims_Usecase{true, true, true}),
+                                            ::testing::Values(cugraph::test::Rmat_Usecase(
+                                              10, 16, 0.57, 0.19, 0.19, 0, false, false))));
+
+INSTANTIATE_TEST_SUITE_P(
+  rmat_benchmark_test, /* note that scale & edge factor can be overridden in benchmarking (with
+                          --gtest_filter to select only the rmat_benchmark_test with a specific
+                          vertex & edge type combination) by command line arguments and do not
+                          include more than one Rmat_Usecase that differ only in scale or edge
+                          factor (to avoid running same benchmarks more than once) */
+  Tests_MGPerVTransformReduceDstKeyAggregatedOutgoingE_Rmat,
+  ::testing::Combine(
+    ::testing::Values(Prims_Usecase{false, false, false},
+                      Prims_Usecase{false, true, false},
+                      Prims_Usecase{true, false, false},
+                      Prims_Usecase{true, true, false}),
+    ::testing::Values(cugraph::test::Rmat_Usecase(20, 32, 0.57, 0.19, 0.19, 0, false, false))));
+
+CUGRAPH_MG_TEST_PROGRAM_MAIN()
diff --git a/cpp/tests/prims/mg_per_v_transform_reduce_incoming_outgoing_e.cu b/cpp/tests/prims/mg_per_v_transform_reduce_incoming_outgoing_e.cu
index efab53f89e6..a459a677569 100644
--- a/cpp/tests/prims/mg_per_v_transform_reduce_incoming_outgoing_e.cu
+++ b/cpp/tests/prims/mg_per_v_transform_reduce_incoming_outgoing_e.cu
@@ -18,6 +18,7 @@
 #include "prims/reduce_op.cuh"
 #include "prims/update_edge_src_dst_property.cuh"
 #include "property_generator.cuh"
+#include "result_compare.cuh"
 #include "utilities/base_fixture.hpp"
 #include "utilities/device_comm_wrapper.hpp"
 #include "utilities/mg_utilities.hpp"
@@ -72,83 +73,6 @@ struct e_op_t {
   }
 };
 
-template <typename T>
-__host__ __device__ bool compare_scalar(T val0, T val1, thrust::optional<T> threshold_ratio)
-{
-  if (threshold_ratio) {
-    return std::abs(val0 - val1) <= (std::max(std::abs(val0), std::abs(val1)) * *threshold_ratio);
-  } else {
-    return val0 == val1;
-  }
-}
-
-template <typename T>
-struct comparator {
-  static constexpr double threshold_ratio{1e-2};
-
-  __host__ __device__ bool operator()(T t0, T t1) const
-  {
-    static_assert(cugraph::is_arithmetic_or_thrust_tuple_of_arithmetic<T>::value);
-    if constexpr (std::is_arithmetic_v<T>) {
-      return compare_scalar(
-        t0,
-        t1,
-        std::is_floating_point_v<T> ? thrust::optional<T>{threshold_ratio} : thrust::nullopt);
-    } else {
-      auto val0   = thrust::get<0>(t0);
-      auto val1   = thrust::get<0>(t1);
-      auto passed = compare_scalar(val0,
-                                   val1,
-                                   std::is_floating_point_v<decltype(val0)>
-                                     ? thrust::optional<decltype(val0)>{threshold_ratio}
-                                     : thrust::nullopt);
-      if (!passed) return false;
-
-      if constexpr (thrust::tuple_size<T>::value >= 2) {
-        auto val0   = thrust::get<1>(t0);
-        auto val1   = thrust::get<1>(t1);
-        auto passed = compare_scalar(val0,
-                                     val1,
-                                     std::is_floating_point_v<decltype(val1)>
-                                       ? thrust::optional<decltype(val1)>{threshold_ratio}
-                                       : thrust::nullopt);
-        if (!passed) return false;
-      }
-      if constexpr (thrust::tuple_size<T>::value >= 3) {
-        assert(false);  // should not be reached.
-      }
-      return true;
-    }
-  }
-};
-
-struct result_compare {
-  const raft::handle_t& handle_;
-  result_compare(raft::handle_t const& handle) : handle_(handle) {}
-
-  template <typename... Args>
-  auto operator()(const std::tuple<rmm::device_uvector<Args>...>& t1,
-                  const std::tuple<rmm::device_uvector<Args>...>& t2)
-  {
-    using type = thrust::tuple<Args...>;
-    return equality_impl(t1, t2, std::make_index_sequence<thrust::tuple_size<type>::value>());
-  }
-
-  template <typename T>
-  auto operator()(const rmm::device_uvector<T>& t1, const rmm::device_uvector<T>& t2)
-  {
-    return thrust::equal(
-      handle_.get_thrust_policy(), t1.begin(), t1.end(), t2.begin(), comparator<T>());
-  }
-
- private:
-  template <typename T, std::size_t... I>
-  auto equality_impl(T& t1, T& t2, std::index_sequence<I...>)
-  {
-    return (... && (result_compare::operator()(std::get<I>(t1), std::get<I>(t2))));
-  }
-};
-
 struct Prims_Usecase {
   bool test_weighted{false};
   bool edge_masking{false};
@@ -440,7 +364,7 @@ class Tests_MGPerVTransformReduceIncomingOutgoingE
             *handle_, sg_graph_view, sg_vertex_prop);
           auto sg_dst_prop = cugraph::test::generate<vertex_t, result_t>::dst_property(
             *handle_, sg_graph_view, sg_vertex_prop);
-          result_compare comp{*handle_};
+          cugraph::test::vector_result_compare compare{*handle_};
 
           auto global_in_result = cugraph::allocate_dataframe_buffer<result_t>(
             sg_graph_view.local_vertex_partition_range_size(), handle_->get_stream());
@@ -528,8 +452,8 @@ class Tests_MGPerVTransformReduceIncomingOutgoingE
             default: FAIL() << "should not be reached.";
           }
 
-          ASSERT_TRUE(comp(mg_aggregate_in_results, global_in_result));
-          ASSERT_TRUE(comp(mg_aggregate_out_results, global_out_result));
+          ASSERT_TRUE(compare(mg_aggregate_in_results, global_in_result));
+          ASSERT_TRUE(compare(mg_aggregate_out_results, global_out_result));
         }
       }
     }
diff --git a/cpp/tests/prims/mg_reduce_v.cu b/cpp/tests/prims/mg_reduce_v.cu
index da3354b77d9..783e17b6d8f 100644
--- a/cpp/tests/prims/mg_reduce_v.cu
+++ b/cpp/tests/prims/mg_reduce_v.cu
@@ -17,6 +17,7 @@
 #include "prims/property_op_utils.cuh"
 #include "prims/reduce_v.cuh"
 #include "property_generator.cuh"
+#include "result_compare.cuh"
 #include "utilities/base_fixture.hpp"
 #include "utilities/device_comm_wrapper.hpp"
 #include "utilities/mg_utilities.hpp"
@@ -49,50 +50,6 @@
 
 #include <random>
 
-template <typename T>
-struct result_compare {
-  static constexpr double threshold_ratio{1e-2};
-  constexpr auto operator()(const T& t1, const T& t2)
-  {
-    if constexpr (std::is_floating_point_v<T>) {
-      bool passed = (t1 == t2)  // when t1 == t2 == 0
-                    ||
-                    (std::abs(t1 - t2) < (std::max(std::abs(t1), std::abs(t2)) * threshold_ratio));
-      return passed;
-    }
-    return t1 == t2;
-  }
-};
-
-template <typename... Args>
-struct result_compare<thrust::tuple<Args...>> {
-  static constexpr double threshold_ratio{1e-3};
-
-  using Type = thrust::tuple<Args...>;
-  constexpr auto operator()(const Type& t1, const Type& t2)
-  {
-    return equality_impl(t1, t2, std::make_index_sequence<thrust::tuple_size<Type>::value>());
-  }
-
- private:
-  template <typename T>
-  constexpr bool equal(T t1, T t2)
-  {
-    if constexpr (std::is_floating_point_v<T>) {
-      bool passed = (t1 == t2)  // when t1 == t2 == 0
-                    ||
-                    (std::abs(t1 - t2) < (std::max(std::abs(t1), std::abs(t2)) * threshold_ratio));
-      return passed;
-    }
-    return t1 == t2;
-  }
-  template <typename T, std::size_t... I>
-  constexpr auto equality_impl(T& t1, T& t2, std::index_sequence<I...>)
-  {
-    return (... && (equal(thrust::get<I>(t1), thrust::get<I>(t2))));
-  }
-};
-
 struct Prims_Usecase {
   bool check_correctness{true};
 };
@@ -249,7 +206,7 @@ class Tests_MGReduceV
               break;
             default: FAIL() << "should not be reached.";
           }
-          result_compare<result_t> compare{};
+          cugraph::test::scalar_result_compare compare{};
           ASSERT_TRUE(compare(expected_result, results[reduction_type]));
         }
       }
diff --git a/cpp/tests/prims/mg_transform_reduce_dst_nbr_intersection_of_e_endpoints_by_v.cu b/cpp/tests/prims/mg_transform_reduce_dst_nbr_intersection_of_e_endpoints_by_v.cu
new file mode 100644
index 00000000000..5fa37250e21
--- /dev/null
+++ b/cpp/tests/prims/mg_transform_reduce_dst_nbr_intersection_of_e_endpoints_by_v.cu
@@ -0,0 +1,289 @@
+/*
+ * Copyright (c) 2024, NVIDIA CORPORATION.
+ *
+ * Licensed 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.
+ */
+
+#include "property_generator.cuh"
+
+#include <cugraph/detail/shuffle_wrappers.hpp>
+#include <cugraph/edge_property.hpp>
+#include <cugraph/edge_src_dst_property.hpp>
+#include <cugraph/graph_functions.hpp>
+#include <cugraph/graph_view.hpp>
+#include <cugraph/utilities/dataframe_buffer.hpp>
+#include <cugraph/utilities/high_res_timer.hpp>
+#include <cugraph/utilities/host_scalar_comm.hpp>
+#include <cugraph/utilities/thrust_tuple_utils.hpp>
+
+#include <raft/comms/mpi_comms.hpp>
+#include <raft/core/comms.hpp>
+#include <raft/core/device_span.hpp>
+#include <raft/core/handle.hpp>
+
+#include <rmm/device_uvector.hpp>
+
+#include <thrust/iterator/counting_iterator.h>
+#include <thrust/tuple.h>
+
+#include <gtest/gtest.h>
+#include <prims/transform_e.cuh>
+#include <prims/transform_reduce_dst_nbr_intersection_of_e_endpoints_by_v.cuh>
+#include <prims/update_edge_src_dst_property.cuh>
+#include <utilities/base_fixture.hpp>
+#include <utilities/device_comm_wrapper.hpp>
+#include <utilities/mg_utilities.hpp>
+#include <utilities/test_graphs.hpp>
+#include <utilities/test_utilities.hpp>
+
+#include <random>
+
+template <typename vertex_t, typename edge_t>
+struct intersection_op_t {
+  __device__ thrust::tuple<edge_t, edge_t, edge_t> operator()(
+    vertex_t v0,
+    vertex_t v1,
+    edge_t v0_prop,
+    edge_t v1_prop,
+    raft::device_span<vertex_t const> intersection) const
+  {
+    return thrust::make_tuple(
+      v0_prop + v1_prop, v0_prop + v1_prop, static_cast<edge_t>(intersection.size()));
+  }
+};
+
+struct Prims_Usecase {
+  bool edge_masking{false};
+  bool check_correctness{true};
+};
+
+template <typename input_usecase_t>
+class Tests_MGTransformReduceDstNbrIntersectionOfEEndpointsByV
+  : public ::testing::TestWithParam<std::tuple<Prims_Usecase, input_usecase_t>> {
+ public:
+  Tests_MGTransformReduceDstNbrIntersectionOfEEndpointsByV() {}
+
+  static void SetUpTestCase() { handle_ = cugraph::test::initialize_mg_handle(); }
+
+  static void TearDownTestCase() { handle_.reset(); }
+
+  virtual void SetUp() {}
+  virtual void TearDown() {}
+
+  // Verify the results of transform_reduce_dst_nbr_intersection_of_e_endpoints_by_v primitive
+  template <typename vertex_t, typename edge_t, typename weight_t>
+  void run_current_test(Prims_Usecase const& prims_usecase, input_usecase_t const& input_usecase)
+  {
+    HighResTimer hr_timer{};
+
+    auto const comm_rank = handle_->get_comms().get_rank();
+    auto const comm_size = handle_->get_comms().get_size();
+
+    // 1. create MG graph
+
+    if (cugraph::test::g_perf) {
+      RAFT_CUDA_TRY(cudaDeviceSynchronize());  // for consistent performance measurement
+      handle_->get_comms().barrier();
+      hr_timer.start("MG Construct graph");
+    }
+
+    cugraph::graph_t<vertex_t, edge_t, false, true> mg_graph(*handle_);
+    std::optional<rmm::device_uvector<vertex_t>> mg_renumber_map{std::nullopt};
+    std::tie(mg_graph, std::ignore, mg_renumber_map) =
+      cugraph::test::construct_graph<vertex_t, edge_t, weight_t, false, true>(
+        *handle_, input_usecase, false, true);
+
+    if (cugraph::test::g_perf) {
+      RAFT_CUDA_TRY(cudaDeviceSynchronize());  // for consistent performance measurement
+      handle_->get_comms().barrier();
+      hr_timer.stop();
+      hr_timer.display_and_clear(std::cout);
+    }
+
+    auto mg_graph_view = mg_graph.view();
+
+    std::optional<cugraph::edge_property_t<decltype(mg_graph_view), bool>> edge_mask{std::nullopt};
+    if (prims_usecase.edge_masking) {
+      edge_mask =
+        cugraph::test::generate<vertex_t, bool>::edge_property(*handle_, mg_graph_view, 2);
+      mg_graph_view.attach_edge_mask((*edge_mask).view());
+    }
+
+    // 2. run MG transform_reduce_dst_nbr_intersection_of_e_endpoints_by_v primitive
+
+    const int hash_bin_count = 5;
+    const int initial_value  = 4;
+
+    auto property_initial_value =
+      cugraph::test::generate<vertex_t, edge_t>::initial_value(initial_value);
+
+    auto mg_vertex_prop = cugraph::test::generate<vertex_t, edge_t>::vertex_property(
+      *handle_, *mg_renumber_map, hash_bin_count);
+    auto mg_src_prop = cugraph::test::generate<vertex_t, edge_t>::src_property(
+      *handle_, mg_graph_view, mg_vertex_prop);
+    auto mg_dst_prop = cugraph::test::generate<vertex_t, edge_t>::dst_property(
+      *handle_, mg_graph_view, mg_vertex_prop);
+
+    auto mg_result_buffer = rmm::device_uvector<edge_t>(
+      mg_graph_view.local_vertex_partition_range_size(), handle_->get_stream());
+
+    if (cugraph::test::g_perf) {
+      RAFT_CUDA_TRY(cudaDeviceSynchronize());  // for consistent performance measurement
+      handle_->get_comms().barrier();
+      hr_timer.start("MG transform_reduce_dst_nbr_intersection_of_e_endpoints_by_v");
+    }
+
+    cugraph::transform_reduce_dst_nbr_intersection_of_e_endpoints_by_v(
+      *handle_,
+      mg_graph_view,
+      mg_src_prop.view(),
+      mg_dst_prop.view(),
+      intersection_op_t<vertex_t, edge_t>{},
+      property_initial_value,
+      mg_result_buffer.begin());
+
+    if (cugraph::test::g_perf) {
+      RAFT_CUDA_TRY(cudaDeviceSynchronize());  // for consistent performance measurement
+      handle_->get_comms().barrier();
+      hr_timer.stop();
+      hr_timer.display_and_clear(std::cout);
+    }
+
+    // 3. validate MG results
+
+    if (prims_usecase.check_correctness) {
+      rmm::device_uvector<edge_t> mg_aggregate_result_buffer(0, handle_->get_stream());
+      std::tie(std::ignore, mg_aggregate_result_buffer) =
+        cugraph::test::mg_vertex_property_values_to_sg_vertex_property_values(
+          *handle_,
+          std::make_optional<raft::device_span<vertex_t const>>((*mg_renumber_map).data(),
+                                                                (*mg_renumber_map).size()),
+          mg_graph_view.local_vertex_partition_range(),
+          std::optional<raft::device_span<vertex_t const>>{std::nullopt},
+          std::optional<raft::device_span<vertex_t const>>{std::nullopt},
+          raft::device_span<edge_t const>(mg_result_buffer.data(), mg_result_buffer.size()));
+
+      cugraph::graph_t<vertex_t, edge_t, false, false> sg_graph(*handle_);
+      std::tie(sg_graph, std::ignore, std::ignore) = cugraph::test::mg_graph_to_sg_graph(
+        *handle_,
+        mg_graph_view,
+        std::optional<cugraph::edge_property_view_t<edge_t, weight_t const*>>{std::nullopt},
+        std::make_optional<raft::device_span<vertex_t const>>((*mg_renumber_map).data(),
+                                                              (*mg_renumber_map).size()),
+        false);
+
+      if (handle_->get_comms().get_rank() == 0) {
+        auto sg_graph_view = sg_graph.view();
+
+        auto sg_vertex_prop = cugraph::test::generate<vertex_t, edge_t>::vertex_property(
+          *handle_,
+          thrust::make_counting_iterator(sg_graph_view.local_vertex_partition_range_first()),
+          thrust::make_counting_iterator(sg_graph_view.local_vertex_partition_range_last()),
+          hash_bin_count);
+        auto sg_src_prop = cugraph::test::generate<vertex_t, edge_t>::src_property(
+          *handle_, sg_graph_view, sg_vertex_prop);
+        auto sg_dst_prop = cugraph::test::generate<vertex_t, edge_t>::dst_property(
+          *handle_, sg_graph_view, sg_vertex_prop);
+
+        auto sg_result_buffer = cugraph::allocate_dataframe_buffer<edge_t>(
+          sg_graph_view.number_of_vertices(), handle_->get_stream());
+
+        cugraph::transform_reduce_dst_nbr_intersection_of_e_endpoints_by_v(
+          *handle_,
+          sg_graph_view,
+          sg_src_prop.view(),
+          sg_dst_prop.view(),
+          intersection_op_t<vertex_t, edge_t>{},
+          property_initial_value,
+          sg_result_buffer.begin());
+
+        bool valid = thrust::equal(handle_->get_thrust_policy(),
+                                   mg_aggregate_result_buffer.begin(),
+                                   mg_aggregate_result_buffer.end(),
+                                   sg_result_buffer.begin());
+
+        ASSERT_TRUE(valid);
+      }
+    }
+  }
+
+ private:
+  static std::unique_ptr<raft::handle_t> handle_;
+};
+
+template <typename input_usecase_t>
+std::unique_ptr<raft::handle_t>
+  Tests_MGTransformReduceDstNbrIntersectionOfEEndpointsByV<input_usecase_t>::handle_ = nullptr;
+
+using Tests_MGTransformReduceDstNbrIntersectionOfEEndpointsByV_File =
+  Tests_MGTransformReduceDstNbrIntersectionOfEEndpointsByV<cugraph::test::File_Usecase>;
+using Tests_MGTransformReduceDstNbrIntersectionOfEEndpointsByV_Rmat =
+  Tests_MGTransformReduceDstNbrIntersectionOfEEndpointsByV<cugraph::test::Rmat_Usecase>;
+
+TEST_P(Tests_MGTransformReduceDstNbrIntersectionOfEEndpointsByV_File, CheckInt32Int32Float)
+{
+  auto param = GetParam();
+  run_current_test<int32_t, int32_t, float>(std::get<0>(param), std::get<1>(param));
+}
+
+TEST_P(Tests_MGTransformReduceDstNbrIntersectionOfEEndpointsByV_Rmat, CheckInt32Int32Float)
+{
+  auto param = GetParam();
+  run_current_test<int32_t, int32_t, float>(
+    std::get<0>(param),
+    cugraph::test::override_Rmat_Usecase_with_cmd_line_arguments(std::get<1>(param)));
+}
+
+TEST_P(Tests_MGTransformReduceDstNbrIntersectionOfEEndpointsByV_Rmat, CheckInt32Int64Float)
+{
+  auto param = GetParam();
+  run_current_test<int32_t, int64_t, float>(
+    std::get<0>(param),
+    cugraph::test::override_Rmat_Usecase_with_cmd_line_arguments(std::get<1>(param)));
+}
+
+TEST_P(Tests_MGTransformReduceDstNbrIntersectionOfEEndpointsByV_Rmat, CheckInt64Int64Float)
+{
+  auto param = GetParam();
+  run_current_test<int64_t, int64_t, float>(
+    std::get<0>(param),
+    cugraph::test::override_Rmat_Usecase_with_cmd_line_arguments(std::get<1>(param)));
+}
+
+INSTANTIATE_TEST_SUITE_P(
+  file_test,
+  Tests_MGTransformReduceDstNbrIntersectionOfEEndpointsByV_File,
+  ::testing::Combine(
+    ::testing::Values(Prims_Usecase{false, true}, Prims_Usecase{true, true}),
+    ::testing::Values(cugraph::test::File_Usecase("test/datasets/karate.mtx"),
+                      cugraph::test::File_Usecase("test/datasets/netscience.mtx"))));
+
+INSTANTIATE_TEST_SUITE_P(rmat_small_test,
+                         Tests_MGTransformReduceDstNbrIntersectionOfEEndpointsByV_Rmat,
+                         ::testing::Combine(::testing::Values(Prims_Usecase{false, true},
+                                                              Prims_Usecase{true, true}),
+                                            ::testing::Values(cugraph::test::Rmat_Usecase(
+                                              10, 16, 0.57, 0.19, 0.19, 0, false, false))));
+
+INSTANTIATE_TEST_SUITE_P(
+  rmat_benchmark_test, /* note that scale & edge factor can be overridden in benchmarking (with
+                          --gtest_filter to select only the rmat_benchmark_test with a specific
+                          vertex & edge type combination) by command line arguments and do not
+                          include more than one Rmat_Usecase that differ only in scale or edge
+                          factor (to avoid running same benchmarks more than once) */
+  Tests_MGTransformReduceDstNbrIntersectionOfEEndpointsByV_Rmat,
+  ::testing::Combine(
+    ::testing::Values(Prims_Usecase{false, false}, Prims_Usecase{true, false}),
+    ::testing::Values(cugraph::test::Rmat_Usecase(20, 32, 0.57, 0.19, 0.19, 0, false, false))));
+
+CUGRAPH_MG_TEST_PROGRAM_MAIN()
diff --git a/cpp/tests/prims/mg_transform_reduce_e.cu b/cpp/tests/prims/mg_transform_reduce_e.cu
index c8ce9fc3a47..53f37e83b30 100644
--- a/cpp/tests/prims/mg_transform_reduce_e.cu
+++ b/cpp/tests/prims/mg_transform_reduce_e.cu
@@ -17,6 +17,7 @@
 #include "prims/transform_reduce_e.cuh"
 #include "prims/update_edge_src_dst_property.cuh"
 #include "property_generator.cuh"
+#include "result_compare.cuh"
 #include "utilities/base_fixture.hpp"
 #include "utilities/device_comm_wrapper.hpp"
 #include "utilities/mg_utilities.hpp"
@@ -52,44 +53,6 @@
 
 #include <random>
 
-template <typename T>
-struct result_compare {
-  static constexpr double threshold_ratio{1e-3};
-  constexpr auto operator()(const T& t1, const T& t2)
-  {
-    if constexpr (std::is_floating_point_v<T>) {
-      return std::abs(t1 - t2) < (std::max(t1, t2) * threshold_ratio);
-    }
-    return t1 == t2;
-  }
-};
-
-template <typename... Args>
-struct result_compare<thrust::tuple<Args...>> {
-  static constexpr double threshold_ratio{1e-3};
-
-  using type = thrust::tuple<Args...>;
-  constexpr auto operator()(const type& t1, const type& t2)
-  {
-    return equality_impl(t1, t2, std::make_index_sequence<thrust::tuple_size<type>::value>());
-  }
-
- private:
-  template <typename T>
-  constexpr bool equal(T t1, T t2)
-  {
-    if constexpr (std::is_floating_point_v<T>) {
-      return std::abs(t1 - t2) < (std::max(t1, t2) * threshold_ratio);
-    }
-    return t1 == t2;
-  }
-  template <typename T, std::size_t... I>
-  constexpr auto equality_impl(T& t1, T& t2, std::index_sequence<I...>)
-  {
-    return (... && (equal(thrust::get<I>(t1), thrust::get<I>(t2))));
-  }
-};
-
 struct Prims_Usecase {
   bool test_weighted{false};
   bool edge_masking{false};
@@ -231,7 +194,7 @@ class Tests_MGTransformReduceE
             }
           },
           property_initial_value);
-        result_compare<result_t> compare{};
+        cugraph::test::scalar_result_compare compare{};
         ASSERT_TRUE(compare(expected_result, result));
       }
     }
diff --git a/cpp/tests/prims/mg_transform_reduce_e_by_src_dst_key.cu b/cpp/tests/prims/mg_transform_reduce_e_by_src_dst_key.cu
new file mode 100644
index 00000000000..457e6b5ab93
--- /dev/null
+++ b/cpp/tests/prims/mg_transform_reduce_e_by_src_dst_key.cu
@@ -0,0 +1,495 @@
+/*
+ * Copyright (c) 2024, NVIDIA CORPORATION.
+ *
+ * Licensed 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.
+ */
+
+#include "property_generator.cuh"
+#include "result_compare.cuh"
+
+#include <cugraph/algorithms.hpp>
+#include <cugraph/edge_partition_view.hpp>
+#include <cugraph/edge_src_dst_property.hpp>
+#include <cugraph/graph_view.hpp>
+#include <cugraph/utilities/dataframe_buffer.hpp>
+#include <cugraph/utilities/high_res_timer.hpp>
+
+#include <raft/comms/mpi_comms.hpp>
+#include <raft/core/comms.hpp>
+#include <raft/core/handle.hpp>
+
+#include <rmm/device_scalar.hpp>
+#include <rmm/device_uvector.hpp>
+
+#include <thrust/count.h>
+#include <thrust/distance.h>
+#include <thrust/functional.h>
+#include <thrust/iterator/counting_iterator.h>
+#include <thrust/optional.h>
+#include <thrust/transform.h>
+#include <thrust/tuple.h>
+
+#include <cuco/hash_functions.cuh>
+
+#include <gtest/gtest.h>
+#include <prims/reduce_op.cuh>
+#include <prims/transform_reduce_e_by_src_dst_key.cuh>
+#include <prims/update_edge_src_dst_property.cuh>
+#include <utilities/base_fixture.hpp>
+#include <utilities/device_comm_wrapper.hpp>
+#include <utilities/mg_utilities.hpp>
+#include <utilities/test_graphs.hpp>
+#include <utilities/test_utilities.hpp>
+#include <utilities/thrust_wrapper.hpp>
+
+#include <random>
+
+struct Prims_Usecase {
+  bool test_weighted{false};
+  bool edge_masking{false};
+  bool check_correctness{true};
+};
+
+template <typename input_usecase_t>
+class Tests_MGTransformReduceEBySrcDstKey
+  : public ::testing::TestWithParam<std::tuple<Prims_Usecase, input_usecase_t>> {
+ public:
+  Tests_MGTransformReduceEBySrcDstKey() {}
+
+  static void SetUpTestCase() { handle_ = cugraph::test::initialize_mg_handle(); }
+
+  static void TearDownTestCase() { handle_.reset(); }
+
+  virtual void SetUp() {}
+  virtual void TearDown() {}
+
+  // Compare the results of transform_reduce_e_by_src|dst_key primitive
+  template <typename vertex_t,
+            typename edge_t,
+            typename weight_t,
+            typename result_t,
+            bool store_transposed>
+  void run_current_test(Prims_Usecase const& prims_usecase, input_usecase_t const& input_usecase)
+  {
+    HighResTimer hr_timer{};
+
+    // 1. create MG graph
+
+    if (cugraph::test::g_perf) {
+      RAFT_CUDA_TRY(cudaDeviceSynchronize());  // for consistent performance measurement
+      handle_->get_comms().barrier();
+      hr_timer.start("MG Construct graph");
+    }
+
+    cugraph::graph_t<vertex_t, edge_t, store_transposed, true> mg_graph(*handle_);
+    std::optional<rmm::device_uvector<vertex_t>> mg_renumber_map{std::nullopt};
+    std::tie(mg_graph, std::ignore, mg_renumber_map) =
+      cugraph::test::construct_graph<vertex_t, edge_t, weight_t, store_transposed, true>(
+        *handle_, input_usecase, prims_usecase.test_weighted, true);
+
+    if (cugraph::test::g_perf) {
+      RAFT_CUDA_TRY(cudaDeviceSynchronize());  // for consistent performance measurement
+      handle_->get_comms().barrier();
+      hr_timer.stop();
+      hr_timer.display_and_clear(std::cout);
+    }
+
+    auto mg_graph_view = mg_graph.view();
+
+    std::optional<cugraph::edge_property_t<decltype(mg_graph_view), bool>> edge_mask{std::nullopt};
+    if (prims_usecase.edge_masking) {
+      edge_mask =
+        cugraph::test::generate<vertex_t, bool>::edge_property(*handle_, mg_graph_view, 2);
+      mg_graph_view.attach_edge_mask((*edge_mask).view());
+    }
+
+    // 2. run MG transform reduce
+
+    const int hash_bin_count = 5;
+    const int initial_value  = 4;
+
+    auto property_initial_value =
+      cugraph::test::generate<vertex_t, result_t>::initial_value(initial_value);
+
+    auto mg_vertex_prop = cugraph::test::generate<vertex_t, result_t>::vertex_property(
+      *handle_, *mg_renumber_map, hash_bin_count);
+    auto mg_src_prop = cugraph::test::generate<vertex_t, result_t>::src_property(
+      *handle_, mg_graph_view, mg_vertex_prop);
+    auto mg_dst_prop = cugraph::test::generate<vertex_t, result_t>::dst_property(
+      *handle_, mg_graph_view, mg_vertex_prop);
+
+    auto mg_vertex_key = cugraph::test::generate<vertex_t, vertex_t>::vertex_property(
+      *handle_, *mg_renumber_map, hash_bin_count);
+    auto mg_src_key = cugraph::test::generate<vertex_t, vertex_t>::src_property(
+      *handle_, mg_graph_view, mg_vertex_key);
+    auto mg_dst_key = cugraph::test::generate<vertex_t, vertex_t>::dst_property(
+      *handle_, mg_graph_view, mg_vertex_key);
+
+    if (cugraph::test::g_perf) {
+      RAFT_CUDA_TRY(cudaDeviceSynchronize());  // for consistent performance measurement
+      handle_->get_comms().barrier();
+      hr_timer.start("MG transform_reduce_e_by_src_key");
+    }
+
+    auto [by_src_keys, by_src_values] = transform_reduce_e_by_src_key(
+      *handle_,
+      mg_graph_view,
+      mg_src_prop.view(),
+      mg_dst_prop.view(),
+      cugraph::edge_dummy_property_t{}.view(),
+      mg_src_key.view(),
+      [] __device__(auto src, auto dst, auto src_property, auto dst_property, thrust::nullopt_t) {
+        if (src_property < dst_property) {
+          return src_property;
+        } else {
+          return dst_property;
+        }
+      },
+      property_initial_value,
+      cugraph::reduce_op::plus<result_t>{});
+
+    if (cugraph::test::g_perf) {
+      RAFT_CUDA_TRY(cudaDeviceSynchronize());  // for consistent performance measurement
+      handle_->get_comms().barrier();
+      hr_timer.stop();
+      hr_timer.display_and_clear(std::cout);
+    }
+
+    if (cugraph::test::g_perf) {
+      RAFT_CUDA_TRY(cudaDeviceSynchronize());  // for consistent performance measurement
+      handle_->get_comms().barrier();
+      hr_timer.start("MG transform_reduce_e_by_dst_key");
+    }
+
+    auto [by_dst_keys, by_dst_values] = transform_reduce_e_by_dst_key(
+      *handle_,
+      mg_graph_view,
+      mg_src_prop.view(),
+      mg_dst_prop.view(),
+      cugraph::edge_dummy_property_t{}.view(),
+      mg_dst_key.view(),
+      [] __device__(auto src, auto dst, auto src_property, auto dst_property, thrust::nullopt_t) {
+        if (src_property < dst_property) {
+          return src_property;
+        } else {
+          return dst_property;
+        }
+      },
+      property_initial_value,
+      cugraph::reduce_op::plus<result_t>{});
+
+    if (cugraph::test::g_perf) {
+      RAFT_CUDA_TRY(cudaDeviceSynchronize());  // for consistent performance measurement
+      handle_->get_comms().barrier();
+      hr_timer.stop();
+      hr_timer.display_and_clear(std::cout);
+    }
+
+    // 3. compare SG & MG results
+
+    if (prims_usecase.check_correctness) {
+      auto mg_aggregate_by_src_keys =
+        cugraph::test::device_gatherv(*handle_, by_src_keys.data(), by_src_keys.size());
+      auto mg_aggregate_by_src_values =
+        cugraph::allocate_dataframe_buffer<result_t>(0, handle_->get_stream());
+      if constexpr (std::is_arithmetic_v<result_t>) {
+        mg_aggregate_by_src_values =
+          cugraph::test::device_gatherv(*handle_, by_src_values.data(), by_src_values.size());
+      } else {
+        std::get<0>(mg_aggregate_by_src_values) = cugraph::test::device_gatherv(
+          *handle_, std::get<0>(by_src_values).data(), std::get<0>(by_src_values).size());
+        std::get<1>(mg_aggregate_by_src_values) = cugraph::test::device_gatherv(
+          *handle_, std::get<1>(by_src_values).data(), std::get<1>(by_src_values).size());
+      }
+      thrust::sort_by_key(handle_->get_thrust_policy(),
+                          mg_aggregate_by_src_keys.begin(),
+                          mg_aggregate_by_src_keys.end(),
+                          cugraph::get_dataframe_buffer_begin(mg_aggregate_by_src_values));
+
+      auto mg_aggregate_by_dst_keys =
+        cugraph::test::device_gatherv(*handle_, by_dst_keys.data(), by_dst_keys.size());
+      auto mg_aggregate_by_dst_values =
+        cugraph::allocate_dataframe_buffer<result_t>(0, handle_->get_stream());
+      if constexpr (std::is_arithmetic_v<result_t>) {
+        mg_aggregate_by_dst_values =
+          cugraph::test::device_gatherv(*handle_, by_dst_values.data(), by_dst_values.size());
+      } else {
+        std::get<0>(mg_aggregate_by_dst_values) = cugraph::test::device_gatherv(
+          *handle_, std::get<0>(by_dst_values).data(), std::get<0>(by_dst_values).size());
+        std::get<1>(mg_aggregate_by_dst_values) = cugraph::test::device_gatherv(
+          *handle_, std::get<1>(by_dst_values).data(), std::get<1>(by_dst_values).size());
+      }
+      thrust::sort_by_key(handle_->get_thrust_policy(),
+                          mg_aggregate_by_dst_keys.begin(),
+                          mg_aggregate_by_dst_keys.end(),
+                          cugraph::get_dataframe_buffer_begin(mg_aggregate_by_dst_values));
+
+      cugraph::graph_t<vertex_t, edge_t, store_transposed, false> sg_graph(*handle_);
+      std::tie(sg_graph, std::ignore, std::ignore) = cugraph::test::mg_graph_to_sg_graph(
+        *handle_,
+        mg_graph_view,
+        std::optional<cugraph::edge_property_view_t<edge_t, weight_t const*>>{std::nullopt},
+        std::make_optional<raft::device_span<vertex_t const>>((*mg_renumber_map).data(),
+                                                              (*mg_renumber_map).size()),
+        false);
+
+      if (handle_->get_comms().get_rank() == 0) {
+        auto sg_graph_view = sg_graph.view();
+
+        auto sg_vertex_prop = cugraph::test::generate<vertex_t, result_t>::vertex_property(
+          *handle_,
+          thrust::make_counting_iterator(sg_graph_view.local_vertex_partition_range_first()),
+          thrust::make_counting_iterator(sg_graph_view.local_vertex_partition_range_last()),
+          hash_bin_count);
+        auto sg_src_prop = cugraph::test::generate<vertex_t, result_t>::src_property(
+          *handle_, sg_graph_view, sg_vertex_prop);
+        auto sg_dst_prop = cugraph::test::generate<vertex_t, result_t>::dst_property(
+          *handle_, sg_graph_view, sg_vertex_prop);
+
+        auto sg_vertex_key = cugraph::test::generate<vertex_t, vertex_t>::vertex_property(
+          *handle_,
+          thrust::make_counting_iterator(sg_graph_view.local_vertex_partition_range_first()),
+          thrust::make_counting_iterator(sg_graph_view.local_vertex_partition_range_last()),
+          hash_bin_count);
+        auto sg_src_key = cugraph::test::generate<vertex_t, vertex_t>::src_property(
+          *handle_, sg_graph_view, sg_vertex_key);
+        auto sg_dst_key = cugraph::test::generate<vertex_t, vertex_t>::dst_property(
+          *handle_, sg_graph_view, sg_vertex_key);
+
+        auto [sg_by_src_keys, sg_by_src_values] = transform_reduce_e_by_src_key(
+          *handle_,
+          sg_graph_view,
+          sg_src_prop.view(),
+          sg_dst_prop.view(),
+          cugraph::edge_dummy_property_t{}.view(),
+          sg_src_key.view(),
+          [] __device__(
+            auto src, auto dst, auto src_property, auto dst_property, thrust::nullopt_t) {
+            if (src_property < dst_property) {
+              return src_property;
+            } else {
+              return dst_property;
+            }
+          },
+          property_initial_value,
+          cugraph::reduce_op::plus<result_t>{});
+        thrust::sort_by_key(handle_->get_thrust_policy(),
+                            sg_by_src_keys.begin(),
+                            sg_by_src_keys.end(),
+                            cugraph::get_dataframe_buffer_begin(sg_by_src_values));
+
+        auto [sg_by_dst_keys, sg_by_dst_values] = transform_reduce_e_by_dst_key(
+          *handle_,
+          sg_graph_view,
+          sg_src_prop.view(),
+          sg_dst_prop.view(),
+          cugraph::edge_dummy_property_t{}.view(),
+          sg_dst_key.view(),
+          [] __device__(
+            auto src, auto dst, auto src_property, auto dst_property, thrust::nullopt_t) {
+            if (src_property < dst_property) {
+              return src_property;
+            } else {
+              return dst_property;
+            }
+          },
+          property_initial_value,
+          cugraph::reduce_op::plus<result_t>{});
+        thrust::sort_by_key(handle_->get_thrust_policy(),
+                            sg_by_dst_keys.begin(),
+                            sg_by_dst_keys.end(),
+                            cugraph::get_dataframe_buffer_begin(sg_by_dst_values));
+
+        cugraph::test::vector_result_compare compare{*handle_};
+
+        ASSERT_TRUE(compare(sg_by_src_keys, mg_aggregate_by_src_keys));
+        ASSERT_TRUE(compare(sg_by_src_values, mg_aggregate_by_src_values));
+
+        ASSERT_TRUE(compare(sg_by_dst_keys, mg_aggregate_by_dst_keys));
+        ASSERT_TRUE(compare(sg_by_dst_values, mg_aggregate_by_dst_values));
+      }
+    }
+  }
+
+ private:
+  static std::unique_ptr<raft::handle_t> handle_;
+};
+
+template <typename input_usecase_t>
+std::unique_ptr<raft::handle_t> Tests_MGTransformReduceEBySrcDstKey<input_usecase_t>::handle_ =
+  nullptr;
+
+using Tests_MGTransformReduceEBySrcDstKey_File =
+  Tests_MGTransformReduceEBySrcDstKey<cugraph::test::File_Usecase>;
+using Tests_MGTransformReduceEBySrcDstKey_Rmat =
+  Tests_MGTransformReduceEBySrcDstKey<cugraph::test::Rmat_Usecase>;
+
+TEST_P(Tests_MGTransformReduceEBySrcDstKey_File, CheckInt32Int32FloatTupleIntFloatTransposeFalse)
+{
+  auto param = GetParam();
+  run_current_test<int32_t, int32_t, float, thrust::tuple<int, float>, false>(std::get<0>(param),
+                                                                              std::get<1>(param));
+}
+
+TEST_P(Tests_MGTransformReduceEBySrcDstKey_Rmat, CheckInt32Int32FloatTupleIntFloatTransposeFalse)
+{
+  auto param = GetParam();
+  run_current_test<int32_t, int32_t, float, thrust::tuple<int, float>, false>(
+    std::get<0>(param),
+    cugraph::test::override_Rmat_Usecase_with_cmd_line_arguments(std::get<1>(param)));
+}
+
+TEST_P(Tests_MGTransformReduceEBySrcDstKey_Rmat, CheckInt32Int64FloatTupleIntFloatTransposeFalse)
+{
+  auto param = GetParam();
+  run_current_test<int32_t, int64_t, float, thrust::tuple<int, float>, false>(
+    std::get<0>(param),
+    cugraph::test::override_Rmat_Usecase_with_cmd_line_arguments(std::get<1>(param)));
+}
+
+TEST_P(Tests_MGTransformReduceEBySrcDstKey_Rmat, CheckInt64Int64FloatTupleIntFloatTransposeFalse)
+{
+  auto param = GetParam();
+  run_current_test<int64_t, int64_t, float, thrust::tuple<int, float>, false>(
+    std::get<0>(param),
+    cugraph::test::override_Rmat_Usecase_with_cmd_line_arguments(std::get<1>(param)));
+}
+
+TEST_P(Tests_MGTransformReduceEBySrcDstKey_File, CheckInt32Int32FloatTupleIntFloatTransposeTrue)
+{
+  auto param = GetParam();
+  run_current_test<int32_t, int32_t, float, thrust::tuple<int, float>, true>(std::get<0>(param),
+                                                                             std::get<1>(param));
+}
+
+TEST_P(Tests_MGTransformReduceEBySrcDstKey_Rmat, CheckInt32Int32FloatTupleIntFloatTransposeTrue)
+{
+  auto param = GetParam();
+  run_current_test<int32_t, int32_t, float, thrust::tuple<int, float>, true>(
+    std::get<0>(param),
+    cugraph::test::override_Rmat_Usecase_with_cmd_line_arguments(std::get<1>(param)));
+}
+
+TEST_P(Tests_MGTransformReduceEBySrcDstKey_Rmat, CheckInt32Int64FloatTupleIntFloatTransposeTrue)
+{
+  auto param = GetParam();
+  run_current_test<int32_t, int64_t, float, thrust::tuple<int, float>, true>(
+    std::get<0>(param),
+    cugraph::test::override_Rmat_Usecase_with_cmd_line_arguments(std::get<1>(param)));
+}
+
+TEST_P(Tests_MGTransformReduceEBySrcDstKey_Rmat, CheckInt64Int64FloatTupleIntFloatTransposeTrue)
+{
+  auto param = GetParam();
+  run_current_test<int64_t, int64_t, float, thrust::tuple<int, float>, true>(
+    std::get<0>(param),
+    cugraph::test::override_Rmat_Usecase_with_cmd_line_arguments(std::get<1>(param)));
+}
+
+TEST_P(Tests_MGTransformReduceEBySrcDstKey_File, CheckInt32Int32FloatTransposeFalse)
+{
+  auto param = GetParam();
+  run_current_test<int32_t, int32_t, float, int, false>(std::get<0>(param), std::get<1>(param));
+}
+
+TEST_P(Tests_MGTransformReduceEBySrcDstKey_Rmat, CheckInt32Int32FloatTransposeFalse)
+{
+  auto param = GetParam();
+  run_current_test<int32_t, int32_t, float, int, false>(
+    std::get<0>(param),
+    cugraph::test::override_Rmat_Usecase_with_cmd_line_arguments(std::get<1>(param)));
+}
+
+TEST_P(Tests_MGTransformReduceEBySrcDstKey_Rmat, CheckInt32Int64FloatTransposeFalse)
+{
+  auto param = GetParam();
+  run_current_test<int32_t, int64_t, float, int, false>(
+    std::get<0>(param),
+    cugraph::test::override_Rmat_Usecase_with_cmd_line_arguments(std::get<1>(param)));
+}
+
+TEST_P(Tests_MGTransformReduceEBySrcDstKey_Rmat, CheckInt64Int64FloatTransposeFalse)
+{
+  auto param = GetParam();
+  run_current_test<int64_t, int64_t, float, int, false>(
+    std::get<0>(param),
+    cugraph::test::override_Rmat_Usecase_with_cmd_line_arguments(std::get<1>(param)));
+}
+
+TEST_P(Tests_MGTransformReduceEBySrcDstKey_File, CheckInt32Int32FloatTransposeTrue)
+{
+  auto param = GetParam();
+  run_current_test<int32_t, int32_t, float, int, true>(std::get<0>(param), std::get<1>(param));
+}
+
+TEST_P(Tests_MGTransformReduceEBySrcDstKey_Rmat, CheckInt32Int32FloatTransposeTrue)
+{
+  auto param = GetParam();
+  run_current_test<int32_t, int32_t, float, int, true>(
+    std::get<0>(param),
+    cugraph::test::override_Rmat_Usecase_with_cmd_line_arguments(std::get<1>(param)));
+}
+
+TEST_P(Tests_MGTransformReduceEBySrcDstKey_Rmat, CheckInt32Int64FloatTransposeTrue)
+{
+  auto param = GetParam();
+  run_current_test<int32_t, int64_t, float, int, true>(
+    std::get<0>(param),
+    cugraph::test::override_Rmat_Usecase_with_cmd_line_arguments(std::get<1>(param)));
+}
+
+TEST_P(Tests_MGTransformReduceEBySrcDstKey_Rmat, CheckInt64Int64FloatTransposeTrue)
+{
+  auto param = GetParam();
+  run_current_test<int64_t, int64_t, float, int, true>(
+    std::get<0>(param),
+    cugraph::test::override_Rmat_Usecase_with_cmd_line_arguments(std::get<1>(param)));
+}
+
+INSTANTIATE_TEST_SUITE_P(
+  file_test,
+  Tests_MGTransformReduceEBySrcDstKey_File,
+  ::testing::Combine(
+    ::testing::Values(Prims_Usecase{false, false, true},
+                      Prims_Usecase{false, true, true},
+                      Prims_Usecase{true, false, true},
+                      Prims_Usecase{true, true, true}),
+    ::testing::Values(cugraph::test::File_Usecase("test/datasets/karate.mtx"),
+                      cugraph::test::File_Usecase("test/datasets/web-Google.mtx"),
+                      cugraph::test::File_Usecase("test/datasets/ljournal-2008.mtx"),
+                      cugraph::test::File_Usecase("test/datasets/webbase-1M.mtx"))));
+
+INSTANTIATE_TEST_SUITE_P(rmat_small_test,
+                         Tests_MGTransformReduceEBySrcDstKey_Rmat,
+                         ::testing::Combine(::testing::Values(Prims_Usecase{false, false, true},
+                                                              Prims_Usecase{false, true, true},
+                                                              Prims_Usecase{true, false, true},
+                                                              Prims_Usecase{true, true, true}),
+                                            ::testing::Values(cugraph::test::Rmat_Usecase(
+                                              10, 16, 0.57, 0.19, 0.19, 0, false, false))));
+
+INSTANTIATE_TEST_SUITE_P(
+  rmat_benchmark_test, /* note that scale & edge factor can be overridden in benchmarking (with
+                          --gtest_filter to select only the rmat_benchmark_test with a specific
+                          vertex & edge type combination) by command line arguments and do not
+                          include more than one Rmat_Usecase that differ only in scale or edge
+                          factor (to avoid running same benchmarks more than once) */
+  Tests_MGTransformReduceEBySrcDstKey_Rmat,
+  ::testing::Combine(
+    ::testing::Values(Prims_Usecase{false, false, false},
+                      Prims_Usecase{false, true, false},
+                      Prims_Usecase{true, false, false},
+                      Prims_Usecase{true, true, false}),
+    ::testing::Values(cugraph::test::Rmat_Usecase(20, 32, 0.57, 0.19, 0.19, 0, false, false))));
+
+CUGRAPH_MG_TEST_PROGRAM_MAIN()
diff --git a/cpp/tests/prims/mg_transform_reduce_v.cu b/cpp/tests/prims/mg_transform_reduce_v.cu
index c0d44bc94f1..c954f31d0f9 100644
--- a/cpp/tests/prims/mg_transform_reduce_v.cu
+++ b/cpp/tests/prims/mg_transform_reduce_v.cu
@@ -16,6 +16,7 @@
 
 #include "prims/transform_reduce_v.cuh"
 #include "property_generator.cuh"
+#include "result_compare.cuh"
 #include "utilities/base_fixture.hpp"
 #include "utilities/device_comm_wrapper.hpp"
 #include "utilities/mg_utilities.hpp"
@@ -56,50 +57,6 @@ struct v_op_t {
   }
 };
 
-template <typename T>
-struct result_compare {
-  static constexpr double threshold_ratio{1e-3};
-  constexpr auto operator()(const T& t1, const T& t2)
-  {
-    if constexpr (std::is_floating_point_v<T>) {
-      bool passed = (t1 == t2)  // when t1 == t2 == 0
-                    ||
-                    (std::abs(t1 - t2) < (std::max(std::abs(t1), std::abs(t2)) * threshold_ratio));
-      return passed;
-    }
-    return t1 == t2;
-  }
-};
-
-template <typename... Args>
-struct result_compare<thrust::tuple<Args...>> {
-  static constexpr double threshold_ratio{1e-3};
-
-  using Type = thrust::tuple<Args...>;
-  constexpr auto operator()(const Type& t1, const Type& t2)
-  {
-    return equality_impl(t1, t2, std::make_index_sequence<thrust::tuple_size<Type>::value>());
-  }
-
- private:
-  template <typename T>
-  constexpr bool equal(T t1, T t2)
-  {
-    if constexpr (std::is_floating_point_v<T>) {
-      bool passed = (t1 == t2)  // when t1 == t2 == 0
-                    ||
-                    (std::abs(t1 - t2) < (std::max(std::abs(t1), std::abs(t2)) * threshold_ratio));
-      return passed;
-    }
-    return t1 == t2;
-  }
-  template <typename T, std::size_t... I>
-  constexpr auto equality_impl(T& t1, T& t2, std::index_sequence<I...>)
-  {
-    return (... && (equal(thrust::get<I>(t1), thrust::get<I>(t2))));
-  }
-};
-
 struct Prims_Usecase {
   bool check_correctness{true};
 };
@@ -254,7 +211,7 @@ class Tests_MGTransformReduceV
               break;
             default: FAIL() << "should not be reached.";
           }
-          result_compare<result_t> compare{};
+          cugraph::test::scalar_result_compare compare{};
           ASSERT_TRUE(compare(expected_result, results[reduction_type]));
         }
       }
diff --git a/cpp/tests/prims/result_compare.cuh b/cpp/tests/prims/result_compare.cuh
new file mode 100644
index 00000000000..5a1abb90e3c
--- /dev/null
+++ b/cpp/tests/prims/result_compare.cuh
@@ -0,0 +1,143 @@
+/*
+ * Copyright (c) 2024, NVIDIA CORPORATION.
+ *
+ * Licensed 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.
+ */
+
+#pragma once
+#include <cugraph/utilities/thrust_tuple_utils.hpp>
+
+#include <raft/core/handle.hpp>
+
+#include <thrust/equal.h>
+#include <thrust/optional.h>
+#include <thrust/tuple.h>
+
+#include <algorithm>
+#include <cmath>
+#include <type_traits>
+#include <utility>
+
+namespace cugraph {
+namespace test {
+
+namespace detail {
+
+template <typename T>
+__host__ __device__ bool compare_arithmetic_scalar(T val0,
+                                                   T val1,
+                                                   thrust::optional<T> threshold_ratio)
+{
+  if (threshold_ratio) {
+    return std::abs(val0 - val1) <= (std::max(std::abs(val0), std::abs(val1)) * *threshold_ratio);
+  } else {
+    return val0 == val1;
+  }
+}
+
+}  // namespace detail
+
+template <typename T>
+struct comparator {
+  static constexpr double threshold_ratio{1e-2};
+
+  __host__ __device__ bool operator()(T t0, T t1) const
+  {
+    static_assert(cugraph::is_arithmetic_or_thrust_tuple_of_arithmetic<T>::value);
+    if constexpr (std::is_arithmetic_v<T>) {
+      return detail::compare_arithmetic_scalar(
+        t0,
+        t1,
+        std::is_floating_point_v<T> ? thrust::optional<T>{threshold_ratio} : thrust::nullopt);
+    } else {
+      auto val0   = thrust::get<0>(t0);
+      auto val1   = thrust::get<0>(t1);
+      auto passed = detail::compare_arithmetic_scalar(
+        val0,
+        val1,
+        std::is_floating_point_v<decltype(val0)> ? thrust::optional<decltype(val0)>{threshold_ratio}
+                                                 : thrust::nullopt);
+      if (!passed) return false;
+
+      if constexpr (thrust::tuple_size<T>::value >= 2) {
+        auto val0 = thrust::get<1>(t0);
+        auto val1 = thrust::get<1>(t1);
+        auto passed =
+          detail::compare_arithmetic_scalar(val0,
+                                            val1,
+                                            std::is_floating_point_v<decltype(val1)>
+                                              ? thrust::optional<decltype(val1)>{threshold_ratio}
+                                              : thrust::nullopt);
+        if (!passed) return false;
+      }
+      if constexpr (thrust::tuple_size<T>::value >= 3) {
+        assert(false);  // should not be reached.
+      }
+      return true;
+    }
+  }
+};
+
+struct scalar_result_compare {
+  template <typename... Args>
+  auto operator()(thrust::tuple<Args...> t1, thrust::tuple<Args...> t2)
+  {
+    using type = thrust::tuple<Args...>;
+    return equality_impl(t1, t2, std::make_index_sequence<thrust::tuple_size<type>::value>());
+  }
+
+  template <typename T>
+  auto operator()(T t1, T t2)
+  {
+    comparator<T> comp{};
+    return comp(t1, t2);
+  }
+
+ private:
+  template <typename T, std::size_t... I>
+  auto equality_impl(T t1, T t2, std::index_sequence<I...>)
+  {
+    return (... && (scalar_result_compare::operator()(thrust::get<I>(t1), thrust::get<I>(t2))));
+  }
+};
+
+struct vector_result_compare {
+  const raft::handle_t& handle_;
+
+  vector_result_compare(raft::handle_t const& handle) : handle_(handle) {}
+
+  template <typename... Args>
+  auto operator()(std::tuple<rmm::device_uvector<Args>...> const& t1,
+                  std::tuple<rmm::device_uvector<Args>...> const& t2)
+  {
+    using type = thrust::tuple<Args...>;
+    return equality_impl(t1, t2, std::make_index_sequence<thrust::tuple_size<type>::value>());
+  }
+
+  template <typename T>
+  auto operator()(rmm::device_uvector<T> const& t1, rmm::device_uvector<T> const& t2)
+  {
+    return thrust::equal(
+      handle_.get_thrust_policy(), t1.begin(), t1.end(), t2.begin(), comparator<T>());
+  }
+
+ private:
+  template <typename T, std::size_t... I>
+  auto equality_impl(T& t1, T& t2, std::index_sequence<I...>)
+  {
+    return (... && (vector_result_compare::operator()(std::get<I>(t1), std::get<I>(t2))));
+  }
+};
+
+}  // namespace test
+}  // namespace cugraph

From 6c88281b6a58bac05a2c29e5c9ab6b952d3a38c6 Mon Sep 17 00:00:00 2001
From: Rick Ratzel <3039903+rlratzel@users.noreply.github.com>
Date: Wed, 13 Mar 2024 09:16:46 -0500
Subject: [PATCH 2/5] Adds nx-cugraph benchmarks for APIs added to prior
 releases that were never benchmarked (#4228)

* Adds benchmark for nx-cugraph `pagerank` with a personalization dict (see results below)
* Adds several placeholder benchmarks for other nx-cugraph APIs added to prior releases that were never benchmarked.

![image](https://github.com/rapidsai/cugraph/assets/3039903/4692e2a2-e14a-489d-84f7-772eda6fc316)

Authors:
  - Rick Ratzel (https://github.com/rlratzel)
  - Ralph Liu (https://github.com/nv-rliu)

Approvers:
  - Erik Welch (https://github.com/eriknw)

URL: https://github.com/rapidsai/cugraph/pull/4228
---
 .../nx-cugraph/pytest-based/bench_algos.py    | 112 ++++++++++++++++++
 1 file changed, 112 insertions(+)

diff --git a/benchmarks/nx-cugraph/pytest-based/bench_algos.py b/benchmarks/nx-cugraph/pytest-based/bench_algos.py
index 97eb32e2aaa..3b085a9bfdb 100644
--- a/benchmarks/nx-cugraph/pytest-based/bench_algos.py
+++ b/benchmarks/nx-cugraph/pytest-based/bench_algos.py
@@ -242,6 +242,28 @@ def get_highest_degree_node(graph_obj):
     return max(degrees, key=lambda t: t[1])[0]
 
 
+def build_personalization_dict(pagerank_dict):
+    """
+    Returns a dictionary that can be used as the personalization value for a
+    call to nx.pagerank(). The pagerank_dict passed in is used as the initial
+    source of values for each node, and this function simply treats the list of
+    dict values as two halves (halves A and B) and swaps them so (most if not
+    all) nodes/keys are assigned a different value from the dictionary.
+    """
+    num_half = len(pagerank_dict) // 2
+    A_half_items = list(pagerank_dict.items())[:num_half]
+    B_half_items = list(pagerank_dict.items())[num_half:]
+
+    # Support an odd number of items by initializing with B_half_items, which
+    # will always be one bigger if the number of items is odd. This will leave
+    # the one remainder (in the case of an odd number) unchanged.
+    pers_dict = dict(B_half_items)
+    pers_dict.update({A_half_items[i][0]: B_half_items[i][1] for i in range(num_half)})
+    pers_dict.update({B_half_items[i][0]: A_half_items[i][1] for i in range(num_half)})
+
+    return pers_dict
+
+
 ################################################################################
 # Benchmarks
 def bench_from_networkx(benchmark, graph_obj):
@@ -431,6 +453,26 @@ def bench_pagerank(benchmark, graph_obj, backend_wrapper):
     assert type(result) is dict
 
 
+def bench_pagerank_personalized(benchmark, graph_obj, backend_wrapper):
+    G = get_graph_obj_for_benchmark(graph_obj, backend_wrapper)
+
+    # FIXME: This will run for every combination of inputs, even if the
+    # graph/dataset does not change. Ideally this is run once per
+    # graph/dataset.
+    pagerank_dict = nx.pagerank(G)
+    personalization_dict = build_personalization_dict(pagerank_dict)
+
+    result = benchmark.pedantic(
+        target=backend_wrapper(nx.pagerank),
+        args=(G,),
+        kwargs={"personalization": personalization_dict},
+        rounds=rounds,
+        iterations=iterations,
+        warmup_rounds=warmup_rounds,
+    )
+    assert type(result) is dict
+
+
 def bench_single_source_shortest_path_length(benchmark, graph_obj, backend_wrapper):
     G = get_graph_obj_for_benchmark(graph_obj, backend_wrapper)
     node = get_highest_degree_node(graph_obj)
@@ -804,3 +846,73 @@ def bench_weakly_connected_components(benchmark, graph_obj, backend_wrapper):
         warmup_rounds=warmup_rounds,
     )
     assert type(result) is list
+
+
+@pytest.mark.skip(reason="benchmark not implemented")
+def bench_complete_bipartite_graph(benchmark, graph_obj, backend_wrapper):
+    pass
+
+
+@pytest.mark.skip(reason="benchmark not implemented")
+def bench_connected_components(benchmark, graph_obj, backend_wrapper):
+    pass
+
+
+@pytest.mark.skip(reason="benchmark not implemented")
+def bench_is_connected(benchmark, graph_obj, backend_wrapper):
+    pass
+
+
+@pytest.mark.skip(reason="benchmark not implemented")
+def bench_node_connected_component(benchmark, graph_obj, backend_wrapper):
+    pass
+
+
+@pytest.mark.skip(reason="benchmark not implemented")
+def bench_number_connected_components(benchmark, graph_obj, backend_wrapper):
+    pass
+
+
+@pytest.mark.skip(reason="benchmark not implemented")
+def bench_is_isolate(benchmark, graph_obj, backend_wrapper):
+    pass
+
+
+@pytest.mark.skip(reason="benchmark not implemented")
+def bench_isolates(benchmark, graph_obj, backend_wrapper):
+    pass
+
+
+@pytest.mark.skip(reason="benchmark not implemented")
+def bench_number_of_isolates(benchmark, graph_obj, backend_wrapper):
+    pass
+
+
+@pytest.mark.skip(reason="benchmark not implemented")
+def bench_complement(benchmark, graph_obj, backend_wrapper):
+    pass
+
+
+@pytest.mark.skip(reason="benchmark not implemented")
+def bench_reverse(benchmark, graph_obj, backend_wrapper):
+    pass
+
+
+@pytest.mark.skip(reason="benchmark not implemented")
+def bench_is_arborescence(benchmark, graph_obj, backend_wrapper):
+    pass
+
+
+@pytest.mark.skip(reason="benchmark not implemented")
+def bench_is_branching(benchmark, graph_obj, backend_wrapper):
+    pass
+
+
+@pytest.mark.skip(reason="benchmark not implemented")
+def bench_is_forest(benchmark, graph_obj, backend_wrapper):
+    pass
+
+
+@pytest.mark.skip(reason="benchmark not implemented")
+def bench_is_tree(benchmark, graph_obj, backend_wrapper):
+    pass

From 6b28aefcabbe10a44de38bd1b3e54f3c717dd559 Mon Sep 17 00:00:00 2001
From: Erik Welch <erik.n.welch@gmail.com>
Date: Wed, 13 Mar 2024 09:18:37 -0500
Subject: [PATCH 3/5] nx-cugraph: add more shortest path algorithms (#4199)

This begins by adding more unweighted shortest path algorithms. Next we'll do weighted via `sssp`, then generic.

Note that there are some performance improvements that can be made:
- add bidirectional search between source and target
  - for `bidirectional_shortest_path` and `has_path`
- alternatively, perform `bfs` from `source` until `target` is reached
- run `all_pairs*` in batched groups

Authors:
  - Erik Welch (https://github.com/eriknw)
  - Ralph Liu (https://github.com/nv-rliu)

Approvers:
  - Brad Rees (https://github.com/BradReesWork)
  - Rick Ratzel (https://github.com/rlratzel)
  - Don Acosta (https://github.com/acostadon)

URL: https://github.com/rapidsai/cugraph/pull/4199
---
 python/nx-cugraph/README.md                   |  25 +-
 python/nx-cugraph/_nx_cugraph/__init__.py     |  52 +++-
 python/nx-cugraph/lint.yaml                   |   4 +-
 .../nx_cugraph/algorithms/__init__.py         |   2 +-
 .../algorithms/bipartite/__init__.py          |   1 -
 .../nx_cugraph/algorithms/bipartite/basic.py  |  31 --
 .../algorithms/centrality/eigenvector.py      |   9 +-
 .../nx_cugraph/algorithms/centrality/katz.py  |   9 +-
 .../algorithms/link_analysis/hits_alg.py      |   9 +-
 .../algorithms/link_analysis/pagerank_alg.py  |   7 +-
 .../algorithms/shortest_paths/__init__.py     |   4 +-
 .../algorithms/shortest_paths/generic.py      | 165 ++++++++++
 .../algorithms/shortest_paths/unweighted.py   | 174 ++++++++++-
 .../algorithms/shortest_paths/weighted.py     | 286 ++++++++++++++++++
 python/nx-cugraph/nx_cugraph/interface.py     |  14 +
 python/nx-cugraph/nx_cugraph/utils/misc.py    |  14 +-
 python/nx-cugraph/scripts/update_readme.py    |   0
 17 files changed, 723 insertions(+), 83 deletions(-)
 delete mode 100644 python/nx-cugraph/nx_cugraph/algorithms/bipartite/basic.py
 create mode 100644 python/nx-cugraph/nx_cugraph/algorithms/shortest_paths/generic.py
 create mode 100644 python/nx-cugraph/nx_cugraph/algorithms/shortest_paths/weighted.py
 mode change 100644 => 100755 python/nx-cugraph/scripts/update_readme.py

diff --git a/python/nx-cugraph/README.md b/python/nx-cugraph/README.md
index 8201dc34eb2..1bf310c8c88 100644
--- a/python/nx-cugraph/README.md
+++ b/python/nx-cugraph/README.md
@@ -95,8 +95,6 @@ Below is the list of algorithms that are currently supported in nx-cugraph.
 
 <pre>
 <a href="https://networkx.org/documentation/stable/reference/algorithms/bipartite.html#module-networkx.algorithms.bipartite">bipartite</a>
- ├─ <a href="https://networkx.org/documentation/stable/reference/algorithms/bipartite.html#module-networkx.algorithms.bipartite.basic">basic</a>
- │   └─ <a href="https://networkx.org/documentation/stable/reference/algorithms/generated/networkx.algorithms.bipartite.basic.is_bipartite.html#networkx.algorithms.bipartite.basic.is_bipartite">is_bipartite</a>
  └─ <a href="https://networkx.org/documentation/stable/reference/algorithms/bipartite.html#module-networkx.algorithms.bipartite.generators">generators</a>
      └─ <a href="https://networkx.org/documentation/stable/reference/algorithms/generated/networkx.algorithms.bipartite.generators.complete_bipartite_graph.html#networkx.algorithms.bipartite.generators.complete_bipartite_graph">complete_bipartite_graph</a>
 <a href="https://networkx.org/documentation/stable/reference/algorithms/centrality.html#module-networkx.algorithms.centrality">centrality</a>
@@ -152,9 +150,26 @@ Below is the list of algorithms that are currently supported in nx-cugraph.
  ├─ <a href="https://networkx.org/documentation/stable/reference/algorithms/generated/networkx.algorithms.reciprocity.overall_reciprocity.html#networkx.algorithms.reciprocity.overall_reciprocity">overall_reciprocity</a>
  └─ <a href="https://networkx.org/documentation/stable/reference/algorithms/generated/networkx.algorithms.reciprocity.reciprocity.html#networkx.algorithms.reciprocity.reciprocity">reciprocity</a>
 <a href="https://networkx.org/documentation/stable/reference/algorithms/shortest_paths.html">shortest_paths</a>
- └─ <a href="https://networkx.org/documentation/stable/reference/algorithms/shortest_paths.html#module-networkx.algorithms.shortest_paths.unweighted">unweighted</a>
-     ├─ <a href="https://networkx.org/documentation/stable/reference/algorithms/generated/networkx.algorithms.shortest_paths.unweighted.single_source_shortest_path_length.html#networkx.algorithms.shortest_paths.unweighted.single_source_shortest_path_length">single_source_shortest_path_length</a>
-     └─ <a href="https://networkx.org/documentation/stable/reference/algorithms/generated/networkx.algorithms.shortest_paths.unweighted.single_target_shortest_path_length.html#networkx.algorithms.shortest_paths.unweighted.single_target_shortest_path_length">single_target_shortest_path_length</a>
+ ├─ <a href="https://networkx.org/documentation/stable/reference/algorithms/shortest_paths.html#module-networkx.algorithms.shortest_paths.generic">generic</a>
+ │   ├─ <a href="https://networkx.org/documentation/stable/reference/algorithms/generated/networkx.algorithms.shortest_paths.generic.has_path.html#networkx.algorithms.shortest_paths.generic.has_path">has_path</a>
+ │   ├─ <a href="https://networkx.org/documentation/stable/reference/algorithms/generated/networkx.algorithms.shortest_paths.generic.shortest_path.html#networkx.algorithms.shortest_paths.generic.shortest_path">shortest_path</a>
+ │   └─ <a href="https://networkx.org/documentation/stable/reference/algorithms/generated/networkx.algorithms.shortest_paths.generic.shortest_path_length.html#networkx.algorithms.shortest_paths.generic.shortest_path_length">shortest_path_length</a>
+ ├─ <a href="https://networkx.org/documentation/stable/reference/algorithms/shortest_paths.html#module-networkx.algorithms.shortest_paths.unweighted">unweighted</a>
+ │   ├─ <a href="https://networkx.org/documentation/stable/reference/algorithms/generated/networkx.algorithms.shortest_paths.unweighted.all_pairs_shortest_path.html#networkx.algorithms.shortest_paths.unweighted.all_pairs_shortest_path">all_pairs_shortest_path</a>
+ │   ├─ <a href="https://networkx.org/documentation/stable/reference/algorithms/generated/networkx.algorithms.shortest_paths.unweighted.all_pairs_shortest_path_length.html#networkx.algorithms.shortest_paths.unweighted.all_pairs_shortest_path_length">all_pairs_shortest_path_length</a>
+ │   ├─ <a href="https://networkx.org/documentation/stable/reference/algorithms/generated/networkx.algorithms.shortest_paths.unweighted.bidirectional_shortest_path.html#networkx.algorithms.shortest_paths.unweighted.bidirectional_shortest_path">bidirectional_shortest_path</a>
+ │   ├─ <a href="https://networkx.org/documentation/stable/reference/algorithms/generated/networkx.algorithms.shortest_paths.unweighted.single_source_shortest_path.html#networkx.algorithms.shortest_paths.unweighted.single_source_shortest_path">single_source_shortest_path</a>
+ │   ├─ <a href="https://networkx.org/documentation/stable/reference/algorithms/generated/networkx.algorithms.shortest_paths.unweighted.single_source_shortest_path_length.html#networkx.algorithms.shortest_paths.unweighted.single_source_shortest_path_length">single_source_shortest_path_length</a>
+ │   ├─ <a href="https://networkx.org/documentation/stable/reference/algorithms/generated/networkx.algorithms.shortest_paths.unweighted.single_target_shortest_path.html#networkx.algorithms.shortest_paths.unweighted.single_target_shortest_path">single_target_shortest_path</a>
+ │   └─ <a href="https://networkx.org/documentation/stable/reference/algorithms/generated/networkx.algorithms.shortest_paths.unweighted.single_target_shortest_path_length.html#networkx.algorithms.shortest_paths.unweighted.single_target_shortest_path_length">single_target_shortest_path_length</a>
+ └─ <a href="https://networkx.org/documentation/stable/reference/algorithms/shortest_paths.html#module-networkx.algorithms.shortest_paths.weighted">weighted</a>
+     ├─ <a href="https://networkx.org/documentation/stable/reference/algorithms/generated/networkx.algorithms.shortest_paths.weighted.all_pairs_bellman_ford_path.html#networkx.algorithms.shortest_paths.weighted.all_pairs_bellman_ford_path">all_pairs_bellman_ford_path</a>
+     ├─ <a href="https://networkx.org/documentation/stable/reference/algorithms/generated/networkx.algorithms.shortest_paths.weighted.all_pairs_bellman_ford_path_length.html#networkx.algorithms.shortest_paths.weighted.all_pairs_bellman_ford_path_length">all_pairs_bellman_ford_path_length</a>
+     ├─ <a href="https://networkx.org/documentation/stable/reference/algorithms/generated/networkx.algorithms.shortest_paths.weighted.bellman_ford_path.html#networkx.algorithms.shortest_paths.weighted.bellman_ford_path">bellman_ford_path</a>
+     ├─ <a href="https://networkx.org/documentation/stable/reference/algorithms/generated/networkx.algorithms.shortest_paths.weighted.bellman_ford_path_length.html#networkx.algorithms.shortest_paths.weighted.bellman_ford_path_length">bellman_ford_path_length</a>
+     ├─ <a href="https://networkx.org/documentation/stable/reference/algorithms/generated/networkx.algorithms.shortest_paths.weighted.single_source_bellman_ford.html#networkx.algorithms.shortest_paths.weighted.single_source_bellman_ford">single_source_bellman_ford</a>
+     ├─ <a href="https://networkx.org/documentation/stable/reference/algorithms/generated/networkx.algorithms.shortest_paths.weighted.single_source_bellman_ford_path.html#networkx.algorithms.shortest_paths.weighted.single_source_bellman_ford_path">single_source_bellman_ford_path</a>
+     └─ <a href="https://networkx.org/documentation/stable/reference/algorithms/generated/networkx.algorithms.shortest_paths.weighted.single_source_bellman_ford_path_length.html#networkx.algorithms.shortest_paths.weighted.single_source_bellman_ford_path_length">single_source_bellman_ford_path_length</a>
 <a href="https://networkx.org/documentation/stable/reference/algorithms/traversal.html">traversal</a>
  └─ <a href="https://networkx.org/documentation/stable/reference/algorithms/traversal.html#module-networkx.algorithms.traversal.breadth_first_search">breadth_first_search</a>
      ├─ <a href="https://networkx.org/documentation/stable/reference/algorithms/generated/networkx.algorithms.traversal.breadth_first_search.bfs_edges.html#networkx.algorithms.traversal.breadth_first_search.bfs_edges">bfs_edges</a>
diff --git a/python/nx-cugraph/_nx_cugraph/__init__.py b/python/nx-cugraph/_nx_cugraph/__init__.py
index b2f13d25ff3..bc7f63fcd49 100644
--- a/python/nx-cugraph/_nx_cugraph/__init__.py
+++ b/python/nx-cugraph/_nx_cugraph/__init__.py
@@ -33,15 +33,22 @@
     # "description": "TODO",
     "functions": {
         # BEGIN: functions
+        "all_pairs_bellman_ford_path",
+        "all_pairs_bellman_ford_path_length",
+        "all_pairs_shortest_path",
+        "all_pairs_shortest_path_length",
         "ancestors",
         "average_clustering",
         "barbell_graph",
+        "bellman_ford_path",
+        "bellman_ford_path_length",
         "betweenness_centrality",
         "bfs_edges",
         "bfs_layers",
         "bfs_predecessors",
         "bfs_successors",
         "bfs_tree",
+        "bidirectional_shortest_path",
         "bull_graph",
         "caveman_graph",
         "chvatal_graph",
@@ -70,6 +77,7 @@
         "from_scipy_sparse_array",
         "frucht_graph",
         "generic_bfs_edges",
+        "has_path",
         "heawood_graph",
         "hits",
         "house_graph",
@@ -77,7 +85,6 @@
         "icosahedral_graph",
         "in_degree_centrality",
         "is_arborescence",
-        "is_bipartite",
         "is_branching",
         "is_connected",
         "is_forest",
@@ -110,7 +117,14 @@
         "reciprocity",
         "reverse",
         "sedgewick_maze_graph",
+        "shortest_path",
+        "shortest_path_length",
+        "single_source_bellman_ford",
+        "single_source_bellman_ford_path",
+        "single_source_bellman_ford_path_length",
+        "single_source_shortest_path",
         "single_source_shortest_path_length",
+        "single_target_shortest_path",
         "single_target_shortest_path_length",
         "star_graph",
         "tadpole_graph",
@@ -128,7 +142,11 @@
     },
     "additional_docs": {
         # BEGIN: additional_docs
+        "all_pairs_bellman_ford_path": "Negative cycles are not yet supported. ``NotImplementedError`` will be raised if there are negative edge weights. We plan to support negative edge weights soon. Also, callable ``weight`` argument is not supported.",
+        "all_pairs_bellman_ford_path_length": "Negative cycles are not yet supported. ``NotImplementedError`` will be raised if there are negative edge weights. We plan to support negative edge weights soon. Also, callable ``weight`` argument is not supported.",
         "average_clustering": "Directed graphs and `weight` parameter are not yet supported.",
+        "bellman_ford_path": "Negative cycles are not yet supported. ``NotImplementedError`` will be raised if there are negative edge weights. We plan to support negative edge weights soon. Also, callable ``weight`` argument is not supported.",
+        "bellman_ford_path_length": "Negative cycles are not yet supported. ``NotImplementedError`` will be raised if there are negative edge weights. We plan to support negative edge weights soon. Also, callable ``weight`` argument is not supported.",
         "betweenness_centrality": "`weight` parameter is not yet supported, and RNG with seed may be different.",
         "bfs_edges": "`sort_neighbors` parameter is not yet supported.",
         "bfs_predecessors": "`sort_neighbors` parameter is not yet supported.",
@@ -147,11 +165,28 @@
         "katz_centrality": "`nstart` isn't used (but is checked), and `normalized=False` is not supported.",
         "louvain_communities": "`seed` parameter is currently ignored, and self-loops are not yet supported.",
         "pagerank": "`dangling` parameter is not supported, but it is checked for validity.",
+        "shortest_path": "Negative weights are not yet supported, and method is ununsed.",
+        "shortest_path_length": "Negative weights are not yet supported, and method is ununsed.",
+        "single_source_bellman_ford": "Negative cycles are not yet supported. ``NotImplementedError`` will be raised if there are negative edge weights. We plan to support negative edge weights soon. Also, callable ``weight`` argument is not supported.",
+        "single_source_bellman_ford_path": "Negative cycles are not yet supported. ``NotImplementedError`` will be raised if there are negative edge weights. We plan to support negative edge weights soon. Also, callable ``weight`` argument is not supported.",
+        "single_source_bellman_ford_path_length": "Negative cycles are not yet supported. ``NotImplementedError`` will be raised if there are negative edge weights. We plan to support negative edge weights soon. Also, callable ``weight`` argument is not supported.",
         "transitivity": "Directed graphs are not yet supported.",
         # END: additional_docs
     },
     "additional_parameters": {
         # BEGIN: additional_parameters
+        "all_pairs_bellman_ford_path": {
+            "dtype : dtype or None, optional": "The data type (np.float32, np.float64, or None) to use for the edge weights in the algorithm. If None, then dtype is determined by the edge values.",
+        },
+        "all_pairs_bellman_ford_path_length": {
+            "dtype : dtype or None, optional": "The data type (np.float32, np.float64, or None) to use for the edge weights in the algorithm. If None, then dtype is determined by the edge values.",
+        },
+        "bellman_ford_path": {
+            "dtype : dtype or None, optional": "The data type (np.float32, np.float64, or None) to use for the edge weights in the algorithm. If None, then dtype is determined by the edge values.",
+        },
+        "bellman_ford_path_length": {
+            "dtype : dtype or None, optional": "The data type (np.float32, np.float64, or None) to use for the edge weights in the algorithm. If None, then dtype is determined by the edge values.",
+        },
         "eigenvector_centrality": {
             "dtype : dtype or None, optional": "The data type (np.float32, np.float64, or None) to use for the edge weights in the algorithm. If None, then dtype is determined by the edge values.",
         },
@@ -169,6 +204,21 @@
         "pagerank": {
             "dtype : dtype or None, optional": "The data type (np.float32, np.float64, or None) to use for the edge weights in the algorithm. If None, then dtype is determined by the edge values.",
         },
+        "shortest_path": {
+            "dtype : dtype or None, optional": "The data type (np.float32, np.float64, or None) to use for the edge weights in the algorithm. If None, then dtype is determined by the edge values.",
+        },
+        "shortest_path_length": {
+            "dtype : dtype or None, optional": "The data type (np.float32, np.float64, or None) to use for the edge weights in the algorithm. If None, then dtype is determined by the edge values.",
+        },
+        "single_source_bellman_ford": {
+            "dtype : dtype or None, optional": "The data type (np.float32, np.float64, or None) to use for the edge weights in the algorithm. If None, then dtype is determined by the edge values.",
+        },
+        "single_source_bellman_ford_path": {
+            "dtype : dtype or None, optional": "The data type (np.float32, np.float64, or None) to use for the edge weights in the algorithm. If None, then dtype is determined by the edge values.",
+        },
+        "single_source_bellman_ford_path_length": {
+            "dtype : dtype or None, optional": "The data type (np.float32, np.float64, or None) to use for the edge weights in the algorithm. If None, then dtype is determined by the edge values.",
+        },
         # END: additional_parameters
     },
 }
diff --git a/python/nx-cugraph/lint.yaml b/python/nx-cugraph/lint.yaml
index fdd24861da7..3239fa151d9 100644
--- a/python/nx-cugraph/lint.yaml
+++ b/python/nx-cugraph/lint.yaml
@@ -50,7 +50,7 @@ repos:
       - id: black
       # - id: black-jupyter
   - repo: https://github.com/astral-sh/ruff-pre-commit
-    rev: v0.2.2
+    rev: v0.3.2
     hooks:
       - id: ruff
         args: [--fix-only, --show-fixes]  # --unsafe-fixes]
@@ -77,7 +77,7 @@ repos:
         additional_dependencies: [tomli]
         files: ^(nx_cugraph|docs)/
   - repo: https://github.com/astral-sh/ruff-pre-commit
-    rev: v0.2.2
+    rev: v0.3.2
     hooks:
       - id: ruff
   - repo: https://github.com/pre-commit/pre-commit-hooks
diff --git a/python/nx-cugraph/nx_cugraph/algorithms/__init__.py b/python/nx-cugraph/nx_cugraph/algorithms/__init__.py
index 7aafa85f5b7..b4a10bcf0a1 100644
--- a/python/nx-cugraph/nx_cugraph/algorithms/__init__.py
+++ b/python/nx-cugraph/nx_cugraph/algorithms/__init__.py
@@ -22,7 +22,7 @@
     traversal,
     tree,
 )
-from .bipartite import complete_bipartite_graph, is_bipartite
+from .bipartite import complete_bipartite_graph
 from .centrality import *
 from .cluster import *
 from .components import *
diff --git a/python/nx-cugraph/nx_cugraph/algorithms/bipartite/__init__.py b/python/nx-cugraph/nx_cugraph/algorithms/bipartite/__init__.py
index e028299c675..bfc7f1d4d42 100644
--- a/python/nx-cugraph/nx_cugraph/algorithms/bipartite/__init__.py
+++ b/python/nx-cugraph/nx_cugraph/algorithms/bipartite/__init__.py
@@ -10,5 +10,4 @@
 # 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.
-from .basic import *
 from .generators import *
diff --git a/python/nx-cugraph/nx_cugraph/algorithms/bipartite/basic.py b/python/nx-cugraph/nx_cugraph/algorithms/bipartite/basic.py
deleted file mode 100644
index 46c6b54075b..00000000000
--- a/python/nx-cugraph/nx_cugraph/algorithms/bipartite/basic.py
+++ /dev/null
@@ -1,31 +0,0 @@
-# Copyright (c) 2024, NVIDIA CORPORATION.
-# Licensed 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.
-import cupy as cp
-
-from nx_cugraph.algorithms.cluster import _triangles
-from nx_cugraph.convert import _to_graph
-from nx_cugraph.utils import networkx_algorithm
-
-__all__ = [
-    "is_bipartite",
-]
-
-
-@networkx_algorithm(version_added="24.02", _plc="triangle_count")
-def is_bipartite(G):
-    G = _to_graph(G)
-    # Counting triangles may not be the fastest way to do this, but it is simple.
-    node_ids, triangles, is_single_node = _triangles(
-        G, None, symmetrize="union" if G.is_directed() else None
-    )
-    return int(cp.count_nonzero(triangles)) == 0
diff --git a/python/nx-cugraph/nx_cugraph/algorithms/centrality/eigenvector.py b/python/nx-cugraph/nx_cugraph/algorithms/centrality/eigenvector.py
index 65a8633667a..c32b6fbb708 100644
--- a/python/nx-cugraph/nx_cugraph/algorithms/centrality/eigenvector.py
+++ b/python/nx-cugraph/nx_cugraph/algorithms/centrality/eigenvector.py
@@ -36,17 +36,12 @@ def eigenvector_centrality(
     G, max_iter=100, tol=1.0e-6, nstart=None, weight=None, *, dtype=None
 ):
     """`nstart` parameter is not used, but it is checked for validity."""
-    G = _to_graph(G, weight, np.float32)
+    G = _to_graph(G, weight, 1, np.float32)
     if len(G) == 0:
         raise nx.NetworkXPointlessConcept(
             "cannot compute centrality for the null graph"
         )
-    if dtype is not None:
-        dtype = _get_float_dtype(dtype)
-    elif weight in G.edge_values:
-        dtype = _get_float_dtype(G.edge_values[weight].dtype)
-    else:
-        dtype = np.float32
+    dtype = _get_float_dtype(dtype, graph=G, weight=weight)
     if nstart is not None:
         # Check if given nstart is valid even though we don't use it
         nstart = G._dict_to_nodearray(nstart, dtype=dtype)
diff --git a/python/nx-cugraph/nx_cugraph/algorithms/centrality/katz.py b/python/nx-cugraph/nx_cugraph/algorithms/centrality/katz.py
index 4a0684f72ee..1c6ed61703d 100644
--- a/python/nx-cugraph/nx_cugraph/algorithms/centrality/katz.py
+++ b/python/nx-cugraph/nx_cugraph/algorithms/centrality/katz.py
@@ -49,15 +49,10 @@ def katz_centrality(
         # Redundant with the `_can_run` check below when being dispatched by NetworkX,
         # but we raise here in case this funcion is called directly.
         raise NotImplementedError("normalized=False is not supported.")
-    G = _to_graph(G, weight, np.float32)
+    G = _to_graph(G, weight, 1, np.float32)
     if (N := len(G)) == 0:
         return {}
-    if dtype is not None:
-        dtype = _get_float_dtype(dtype)
-    elif weight in G.edge_values:
-        dtype = _get_float_dtype(G.edge_values[weight].dtype)
-    else:
-        dtype = np.float32
+    dtype = _get_float_dtype(dtype, graph=G, weight=weight)
     if nstart is not None:
         # Check if given nstart is valid even though we don't use it
         nstart = G._dict_to_nodearray(nstart, 0, dtype)
diff --git a/python/nx-cugraph/nx_cugraph/algorithms/link_analysis/hits_alg.py b/python/nx-cugraph/nx_cugraph/algorithms/link_analysis/hits_alg.py
index e61a931c069..e529b83ab1a 100644
--- a/python/nx-cugraph/nx_cugraph/algorithms/link_analysis/hits_alg.py
+++ b/python/nx-cugraph/nx_cugraph/algorithms/link_analysis/hits_alg.py
@@ -46,15 +46,10 @@ def hits(
     weight="weight",
     dtype=None,
 ):
-    G = _to_graph(G, weight, np.float32)
+    G = _to_graph(G, weight, 1, np.float32)
     if (N := len(G)) == 0:
         return {}, {}
-    if dtype is not None:
-        dtype = _get_float_dtype(dtype)
-    elif weight in G.edge_values:
-        dtype = _get_float_dtype(G.edge_values[weight].dtype)
-    else:
-        dtype = np.float32
+    dtype = _get_float_dtype(dtype, graph=G, weight=weight)
     if nstart is not None:
         nstart = G._dict_to_nodearray(nstart, 0, dtype)
     if max_iter <= 0:
diff --git a/python/nx-cugraph/nx_cugraph/algorithms/link_analysis/pagerank_alg.py b/python/nx-cugraph/nx_cugraph/algorithms/link_analysis/pagerank_alg.py
index 40224e91d57..41203a2bc22 100644
--- a/python/nx-cugraph/nx_cugraph/algorithms/link_analysis/pagerank_alg.py
+++ b/python/nx-cugraph/nx_cugraph/algorithms/link_analysis/pagerank_alg.py
@@ -48,12 +48,7 @@ def pagerank(
     G = _to_graph(G, weight, 1, np.float32)
     if (N := len(G)) == 0:
         return {}
-    if dtype is not None:
-        dtype = _get_float_dtype(dtype)
-    elif weight in G.edge_values:
-        dtype = _get_float_dtype(G.edge_values[weight].dtype)
-    else:
-        dtype = np.float32
+    dtype = _get_float_dtype(dtype, graph=G, weight=weight)
     if nstart is not None:
         nstart = G._dict_to_nodearray(nstart, 0, dtype=dtype)
         if (total := nstart.sum()) == 0:
diff --git a/python/nx-cugraph/nx_cugraph/algorithms/shortest_paths/__init__.py b/python/nx-cugraph/nx_cugraph/algorithms/shortest_paths/__init__.py
index b7d6b742176..9d87389a98e 100644
--- a/python/nx-cugraph/nx_cugraph/algorithms/shortest_paths/__init__.py
+++ b/python/nx-cugraph/nx_cugraph/algorithms/shortest_paths/__init__.py
@@ -1,4 +1,4 @@
-# Copyright (c) 2023, NVIDIA CORPORATION.
+# Copyright (c) 2023-2024, NVIDIA CORPORATION.
 # Licensed 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
@@ -10,4 +10,6 @@
 # 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.
+from .generic import *
 from .unweighted import *
+from .weighted import *
diff --git a/python/nx-cugraph/nx_cugraph/algorithms/shortest_paths/generic.py b/python/nx-cugraph/nx_cugraph/algorithms/shortest_paths/generic.py
new file mode 100644
index 00000000000..68dbbace93d
--- /dev/null
+++ b/python/nx-cugraph/nx_cugraph/algorithms/shortest_paths/generic.py
@@ -0,0 +1,165 @@
+# Copyright (c) 2024, NVIDIA CORPORATION.
+# Licensed 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.
+import networkx as nx
+import numpy as np
+
+import nx_cugraph as nxcg
+from nx_cugraph.convert import _to_graph
+from nx_cugraph.utils import _dtype_param, _get_float_dtype, networkx_algorithm
+
+from .unweighted import _bfs
+from .weighted import _sssp
+
+__all__ = [
+    "shortest_path",
+    "shortest_path_length",
+    "has_path",
+]
+
+
+@networkx_algorithm(version_added="24.04", _plc="bfs")
+def has_path(G, source, target):
+    # TODO PERF: make faster in core
+    try:
+        nxcg.bidirectional_shortest_path(G, source, target)
+    except nx.NetworkXNoPath:
+        return False
+    return True
+
+
+@networkx_algorithm(
+    extra_params=_dtype_param, version_added="24.04", _plc={"bfs", "sssp"}
+)
+def shortest_path(
+    G, source=None, target=None, weight=None, method="dijkstra", *, dtype=None
+):
+    """Negative weights are not yet supported, and method is ununsed."""
+    if method not in {"dijkstra", "bellman-ford"}:
+        raise ValueError(f"method not supported: {method}")
+    if weight is None:
+        method = "unweighted"
+    if source is None:
+        if target is None:
+            # All pairs
+            if method == "unweighted":
+                paths = nxcg.all_pairs_shortest_path(G)
+            else:
+                # method == "dijkstra":
+                # method == 'bellman-ford':
+                paths = nxcg.all_pairs_bellman_ford_path(G, weight=weight, dtype=dtype)
+            if nx.__version__[:3] <= "3.4":
+                paths = dict(paths)
+        # To target
+        elif method == "unweighted":
+            paths = nxcg.single_target_shortest_path(G, target)
+        else:
+            # method == "dijkstra":
+            # method == 'bellman-ford':
+            # XXX: it seems weird that `reverse_path=True` is necessary here
+            G = _to_graph(G, weight, 1, np.float32)
+            dtype = _get_float_dtype(dtype, graph=G, weight=weight)
+            paths = _sssp(
+                G, target, weight, return_type="path", dtype=dtype, reverse_path=True
+            )
+    elif target is None:
+        # From source
+        if method == "unweighted":
+            paths = nxcg.single_source_shortest_path(G, source)
+        else:
+            # method == "dijkstra":
+            # method == 'bellman-ford':
+            paths = nxcg.single_source_bellman_ford_path(
+                G, source, weight=weight, dtype=dtype
+            )
+    # From source to target
+    elif method == "unweighted":
+        paths = nxcg.bidirectional_shortest_path(G, source, target)
+    else:
+        # method == "dijkstra":
+        # method == 'bellman-ford':
+        paths = nxcg.bellman_ford_path(G, source, target, weight, dtype=dtype)
+    return paths
+
+
+@shortest_path._can_run
+def _(G, source=None, target=None, weight=None, method="dijkstra", *, dtype=None):
+    return (
+        weight is None
+        or not callable(weight)
+        and not nx.is_negatively_weighted(G, weight=weight)
+    )
+
+
+@networkx_algorithm(
+    extra_params=_dtype_param, version_added="24.04", _plc={"bfs", "sssp"}
+)
+def shortest_path_length(
+    G, source=None, target=None, weight=None, method="dijkstra", *, dtype=None
+):
+    """Negative weights are not yet supported, and method is ununsed."""
+    if method not in {"dijkstra", "bellman-ford"}:
+        raise ValueError(f"method not supported: {method}")
+    if weight is None:
+        method = "unweighted"
+    if source is None:
+        if target is None:
+            # All pairs
+            if method == "unweighted":
+                lengths = nxcg.all_pairs_shortest_path_length(G)
+            else:
+                # method == "dijkstra":
+                # method == 'bellman-ford':
+                lengths = nxcg.all_pairs_bellman_ford_path_length(
+                    G, weight=weight, dtype=dtype
+                )
+        # To target
+        elif method == "unweighted":
+            lengths = nxcg.single_target_shortest_path_length(G, target)
+            if nx.__version__[:3] <= "3.4":
+                lengths = dict(lengths)
+        else:
+            # method == "dijkstra":
+            # method == 'bellman-ford':
+            lengths = nxcg.single_source_bellman_ford_path_length(
+                G, target, weight=weight, dtype=dtype
+            )
+    elif target is None:
+        # From source
+        if method == "unweighted":
+            lengths = nxcg.single_source_shortest_path_length(G, source)
+        else:
+            # method == "dijkstra":
+            # method == 'bellman-ford':
+            lengths = dict(
+                nxcg.single_source_bellman_ford_path_length(
+                    G, source, weight=weight, dtype=dtype
+                )
+            )
+    # From source to target
+    elif method == "unweighted":
+        G = _to_graph(G)
+        lengths = _bfs(G, source, None, "Source", return_type="length", target=target)
+    else:
+        # method == "dijkstra":
+        # method == 'bellman-ford':
+        lengths = nxcg.bellman_ford_path_length(G, source, target, weight, dtype=dtype)
+    return lengths
+
+
+@shortest_path_length._can_run
+def _(G, source=None, target=None, weight=None, method="dijkstra", *, dtype=None):
+    return (
+        weight is None
+        or not callable(weight)
+        and not nx.is_negatively_weighted(G, weight=weight)
+    )
diff --git a/python/nx-cugraph/nx_cugraph/algorithms/shortest_paths/unweighted.py b/python/nx-cugraph/nx_cugraph/algorithms/shortest_paths/unweighted.py
index 2012495953e..714289c5b4b 100644
--- a/python/nx-cugraph/nx_cugraph/algorithms/shortest_paths/unweighted.py
+++ b/python/nx-cugraph/nx_cugraph/algorithms/shortest_paths/unweighted.py
@@ -10,33 +10,127 @@
 # 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.
+import itertools
+
 import cupy as cp
 import networkx as nx
 import numpy as np
 import pylibcugraph as plc
 
 from nx_cugraph.convert import _to_graph
-from nx_cugraph.utils import index_dtype, networkx_algorithm
+from nx_cugraph.utils import _groupby, index_dtype, networkx_algorithm
+
+__all__ = [
+    "bidirectional_shortest_path",
+    "single_source_shortest_path",
+    "single_source_shortest_path_length",
+    "single_target_shortest_path",
+    "single_target_shortest_path_length",
+    "all_pairs_shortest_path",
+    "all_pairs_shortest_path_length",
+]
 
-__all__ = ["single_source_shortest_path_length", "single_target_shortest_path_length"]
+concat = itertools.chain.from_iterable
 
 
 @networkx_algorithm(version_added="23.12", _plc="bfs")
 def single_source_shortest_path_length(G, source, cutoff=None):
-    return _single_shortest_path_length(G, source, cutoff, "Source")
+    G = _to_graph(G)
+    return _bfs(G, source, cutoff, "Source", return_type="length")
 
 
 @networkx_algorithm(version_added="23.12", _plc="bfs")
 def single_target_shortest_path_length(G, target, cutoff=None):
-    return _single_shortest_path_length(G, target, cutoff, "Target")
+    G = _to_graph(G)
+    rv = _bfs(G, target, cutoff, "Target", return_type="length")
+    if nx.__version__[:3] <= "3.4":
+        return iter(rv.items())
+    return rv
+
+
+@networkx_algorithm(version_added="24.04", _plc="bfs")
+def all_pairs_shortest_path_length(G, cutoff=None):
+    # TODO PERF: batched bfs to compute many at once
+    G = _to_graph(G)
+    for n in G:
+        yield (n, _bfs(G, n, cutoff, "Source", return_type="length"))
 
 
-def _single_shortest_path_length(G, source, cutoff, kind):
+@networkx_algorithm(version_added="24.04", _plc="bfs")
+def bidirectional_shortest_path(G, source, target):
+    # TODO PERF: do bidirectional traversal in core
     G = _to_graph(G)
+    if source not in G or target not in G:
+        raise nx.NodeNotFound(f"Either source {source} or target {target} is not in G")
+    return _bfs(G, source, None, "Source", return_type="path", target=target)
+
+
+@networkx_algorithm(version_added="24.04", _plc="bfs")
+def single_source_shortest_path(G, source, cutoff=None):
+    G = _to_graph(G)
+    return _bfs(G, source, cutoff, "Source", return_type="path")
+
+
+@networkx_algorithm(version_added="24.04", _plc="bfs")
+def single_target_shortest_path(G, target, cutoff=None):
+    G = _to_graph(G)
+    return _bfs(G, target, cutoff, "Target", return_type="path", reverse_path=True)
+
+
+@networkx_algorithm(version_added="24.04", _plc="bfs")
+def all_pairs_shortest_path(G, cutoff=None):
+    # TODO PERF: batched bfs to compute many at once
+    G = _to_graph(G)
+    for n in G:
+        yield (n, _bfs(G, n, cutoff, "Source", return_type="path"))
+
+
+def _bfs(
+    G, source, cutoff, kind, *, return_type, reverse_path=False, target=None, scale=None
+):
+    """BFS for unweighted shortest path algorithms.
+
+    Parameters
+    ----------
+    source: node label
+
+    cutoff: int, optional
+
+    kind: {"Source", "Target"}
+
+    return_type: {"length", "path", "length-path"}
+
+    reverse_path: bool
+
+    target: node label
+
+    scale: int or float, optional
+        The amount to scale the lengths
+    """
+    # DRY: _sssp in weighted.py has similar code
     if source not in G:
-        raise nx.NodeNotFound(f"{kind} {source} is not in G")
-    if G.src_indices.size == 0:
-        return {source: 0}
+        # Different message to pass networkx tests
+        if return_type == "length":
+            raise nx.NodeNotFound(f"{kind} {source} is not in G")
+        raise nx.NodeNotFound(f"{kind} {source} not in G")
+    if target is not None:
+        if source == target or cutoff is not None and cutoff <= 0:
+            if return_type == "path":
+                return [source]
+            if return_type == "length":
+                return 0
+            # return_type == "length-path"
+            return 0, [source]
+        if target not in G or G.src_indices.size == 0:
+            raise nx.NetworkXNoPath(f"Node {target} not reachable from {source}")
+    elif G.src_indices.size == 0 or cutoff is not None and cutoff <= 0:
+        if return_type == "path":
+            return {source: [source]}
+        if return_type == "length":
+            return {source: 0}
+        # return_type == "length-path"
+        return {source: 0}, {source: [source]}
+
     if cutoff is None:
         cutoff = -1
     src_index = source if G.key_to_id is None else G.key_to_id[source]
@@ -46,8 +140,68 @@ def _single_shortest_path_length(G, source, cutoff, kind):
         sources=cp.array([src_index], index_dtype),
         direction_optimizing=False,  # True for undirected only; what's recommended?
         depth_limit=cutoff,
-        compute_predecessors=False,
+        compute_predecessors=return_type != "length",
         do_expensive_check=False,
     )
     mask = distances != np.iinfo(distances.dtype).max
-    return G._nodearrays_to_dict(node_ids[mask], distances[mask])
+    node_ids = node_ids[mask]
+    if return_type != "path":
+        lengths = distances = distances[mask]
+        if scale is not None:
+            lengths = scale * lengths
+        lengths = G._nodearrays_to_dict(node_ids, lengths)
+        if target is not None:
+            if target not in lengths:
+                raise nx.NetworkXNoPath(f"Node {target} not reachable from {source}")
+            lengths = lengths[target]
+    if return_type != "length":
+        if target is not None:
+            d = dict(zip(node_ids.tolist(), predecessors[mask].tolist()))
+            dst_index = target if G.key_to_id is None else G.key_to_id[target]
+            if dst_index not in d:
+                raise nx.NetworkXNoPath(f"Node {target} not reachable from {source}")
+            cur = dst_index
+            paths = [dst_index]
+            while cur != src_index:
+                cur = d[cur]
+                paths.append(cur)
+            if (id_to_key := G.id_to_key) is not None:
+                if reverse_path:
+                    paths = [id_to_key[cur] for cur in paths]
+                else:
+                    paths = [id_to_key[cur] for cur in reversed(paths)]
+            elif not reverse_path:
+                paths.reverse()
+        else:
+            if return_type == "path":
+                distances = distances[mask]
+            groups = _groupby(distances, [predecessors[mask], node_ids])
+
+            # `pred_node_iter` does the equivalent as these nested for loops:
+            # for length in range(1, len(groups)):
+            #     preds, nodes = groups[length]
+            #     for pred, node in zip(preds.tolist(), nodes.tolist()):
+            if G.key_to_id is None:
+                pred_node_iter = concat(
+                    zip(*(x.tolist() for x in groups[length]))
+                    for length in range(1, len(groups))
+                )
+            else:
+                pred_node_iter = concat(
+                    zip(*(G._nodeiter_to_iter(x.tolist()) for x in groups[length]))
+                    for length in range(1, len(groups))
+                )
+            # Consider making utility functions for creating paths
+            paths = {source: [source]}
+            if reverse_path:
+                for pred, node in pred_node_iter:
+                    paths[node] = [node, *paths[pred]]
+            else:
+                for pred, node in pred_node_iter:
+                    paths[node] = [*paths[pred], node]
+    if return_type == "path":
+        return paths
+    if return_type == "length":
+        return lengths
+    # return_type == "length-path"
+    return lengths, paths
diff --git a/python/nx-cugraph/nx_cugraph/algorithms/shortest_paths/weighted.py b/python/nx-cugraph/nx_cugraph/algorithms/shortest_paths/weighted.py
new file mode 100644
index 00000000000..32323dd45f3
--- /dev/null
+++ b/python/nx-cugraph/nx_cugraph/algorithms/shortest_paths/weighted.py
@@ -0,0 +1,286 @@
+# Copyright (c) 2024, NVIDIA CORPORATION.
+# Licensed 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.
+import networkx as nx
+import numpy as np
+import pylibcugraph as plc
+
+from nx_cugraph.convert import _to_graph
+from nx_cugraph.utils import (
+    _dtype_param,
+    _get_float_dtype,
+    _groupby,
+    networkx_algorithm,
+)
+
+from .unweighted import _bfs
+
+__all__ = [
+    "bellman_ford_path",
+    "bellman_ford_path_length",
+    "single_source_bellman_ford",
+    "single_source_bellman_ford_path",
+    "single_source_bellman_ford_path_length",
+    "all_pairs_bellman_ford_path",
+    "all_pairs_bellman_ford_path_length",
+]
+
+
+def _add_doc(func):
+    func.__doc__ = (
+        "Negative cycles are not yet supported. ``NotImplementedError`` will be raised "
+        "if there are negative edge weights. We plan to support negative edge weights "
+        "soon. Also, callable ``weight`` argument is not supported."
+    )
+    return func
+
+
+@networkx_algorithm(extra_params=_dtype_param, version_added="24.04", _plc="sssp")
+@_add_doc
+def bellman_ford_path(G, source, target, weight="weight", *, dtype=None):
+    G = _to_graph(G, weight, 1, np.float32)
+    dtype = _get_float_dtype(dtype, graph=G, weight=weight)
+    return _sssp(G, source, weight, target, return_type="path", dtype=dtype)
+
+
+@bellman_ford_path._can_run
+def _(G, source, target, weight="weight", *, dtype=None):
+    return (
+        weight is None
+        or not callable(weight)
+        and not nx.is_negatively_weighted(G, weight=weight)
+    )
+
+
+@networkx_algorithm(extra_params=_dtype_param, version_added="24.04", _plc="sssp")
+@_add_doc
+def bellman_ford_path_length(G, source, target, weight="weight", *, dtype=None):
+    G = _to_graph(G, weight, 1, np.float32)
+    dtype = _get_float_dtype(dtype, graph=G, weight=weight)
+    return _sssp(G, source, weight, target, return_type="length", dtype=dtype)
+
+
+@bellman_ford_path_length._can_run
+def _(G, source, target, weight="weight", *, dtype=None):
+    return (
+        weight is None
+        or not callable(weight)
+        and not nx.is_negatively_weighted(G, weight=weight)
+    )
+
+
+@networkx_algorithm(extra_params=_dtype_param, version_added="24.04", _plc="sssp")
+@_add_doc
+def single_source_bellman_ford_path(G, source, weight="weight", *, dtype=None):
+    G = _to_graph(G, weight, 1, np.float32)
+    dtype = _get_float_dtype(dtype, graph=G, weight=weight)
+    return _sssp(G, source, weight, return_type="path", dtype=dtype)
+
+
+@single_source_bellman_ford_path._can_run
+def _(G, source, weight="weight", *, dtype=None):
+    return (
+        weight is None
+        or not callable(weight)
+        and not nx.is_negatively_weighted(G, weight=weight)
+    )
+
+
+@networkx_algorithm(extra_params=_dtype_param, version_added="24.04", _plc="sssp")
+@_add_doc
+def single_source_bellman_ford_path_length(G, source, weight="weight", *, dtype=None):
+    G = _to_graph(G, weight, 1, np.float32)
+    dtype = _get_float_dtype(dtype, graph=G, weight=weight)
+    return _sssp(G, source, weight, return_type="length", dtype=dtype)
+
+
+@single_source_bellman_ford_path_length._can_run
+def _(G, source, weight="weight", *, dtype=None):
+    return (
+        weight is None
+        or not callable(weight)
+        and not nx.is_negatively_weighted(G, weight=weight)
+    )
+
+
+@networkx_algorithm(extra_params=_dtype_param, version_added="24.04", _plc="sssp")
+@_add_doc
+def single_source_bellman_ford(G, source, target=None, weight="weight", *, dtype=None):
+    G = _to_graph(G, weight, 1, np.float32)
+    dtype = _get_float_dtype(dtype, graph=G, weight=weight)
+    return _sssp(G, source, weight, target, return_type="length-path", dtype=dtype)
+
+
+@single_source_bellman_ford._can_run
+def _(G, source, target=None, weight="weight", *, dtype=None):
+    return (
+        weight is None
+        or not callable(weight)
+        and not nx.is_negatively_weighted(G, weight=weight)
+    )
+
+
+@networkx_algorithm(extra_params=_dtype_param, version_added="24.04", _plc="sssp")
+@_add_doc
+def all_pairs_bellman_ford_path_length(G, weight="weight", *, dtype=None):
+    # TODO PERF: batched bfs to compute many at once
+    G = _to_graph(G, weight, 1, np.float32)
+    dtype = _get_float_dtype(dtype, graph=G, weight=weight)
+    for n in G:
+        yield (n, _sssp(G, n, weight, return_type="length", dtype=dtype))
+
+
+@all_pairs_bellman_ford_path_length._can_run
+def _(G, weight="weight", *, dtype=None):
+    return (
+        weight is None
+        or not callable(weight)
+        and not nx.is_negatively_weighted(G, weight=weight)
+    )
+
+
+@networkx_algorithm(extra_params=_dtype_param, version_added="24.04", _plc="sssp")
+@_add_doc
+def all_pairs_bellman_ford_path(G, weight="weight", *, dtype=None):
+    # TODO PERF: batched bfs to compute many at once
+    G = _to_graph(G, weight, 1, np.float32)
+    dtype = _get_float_dtype(dtype, graph=G, weight=weight)
+    for n in G:
+        yield (n, _sssp(G, n, weight, return_type="path", dtype=dtype))
+
+
+@all_pairs_bellman_ford_path._can_run
+def _(G, weight="weight", *, dtype=None):
+    return (
+        weight is None
+        or not callable(weight)
+        and not nx.is_negatively_weighted(G, weight=weight)
+    )
+
+
+def _sssp(G, source, weight, target=None, *, return_type, dtype, reverse_path=False):
+    """SSSP for weighted shortest paths.
+
+    Parameters
+    ----------
+    return_type : {"length", "path", "length-path"}
+
+    """
+    # DRY: _bfs in unweighted.py has similar code
+    if source not in G:
+        raise nx.NodeNotFound(f"Node {source} not found in graph")
+    if target is not None:
+        if source == target:
+            if return_type == "path":
+                return [source]
+            if return_type == "length":
+                return 0
+            # return_type == "length-path"
+            return 0, [source]
+        if target not in G or G.src_indices.size == 0:
+            raise nx.NetworkXNoPath(f"Node {target} not reachable from {source}")
+    elif G.src_indices.size == 0:
+        if return_type == "path":
+            return {source: [source]}
+        if return_type == "length":
+            return {source: 0}
+        # return_type == "length-path"
+        return {source: 0}, {source: [source]}
+
+    if callable(weight):
+        raise NotImplementedError("callable `weight` argument is not supported")
+
+    if weight not in G.edge_values:
+        # No edge values, so use BFS instead
+        return _bfs(G, source, None, "Source", return_type=return_type, target=target)
+
+    # Check for negative values since we don't support negative cycles
+    edge_vals = G.edge_values[weight]
+    if weight in G.edge_masks:
+        edge_vals = edge_vals[G.edge_masks[weight]]
+    if (edge_vals < 0).any():
+        raise NotImplementedError("Negative edge weights not yet supported")
+    edge_val = edge_vals[0]
+    if (edge_vals == edge_val).all() and (
+        edge_vals.size == G.src_indices.size or edge_val == 1
+    ):
+        # Edge values are all the same, so use scaled BFS instead
+        return _bfs(
+            G,
+            source,
+            None,
+            "Source",
+            return_type=return_type,
+            target=target,
+            scale=edge_val,
+            reverse_path=reverse_path,
+        )
+
+    src_index = source if G.key_to_id is None else G.key_to_id[source]
+    node_ids, distances, predecessors = plc.sssp(
+        resource_handle=plc.ResourceHandle(),
+        graph=G._get_plc_graph(weight, 1, dtype),
+        source=src_index,
+        cutoff=np.inf,
+        compute_predecessors=True,  # TODO: False is not yet supported
+        # compute_predecessors=return_type != "length",
+        do_expensive_check=False,
+    )
+    mask = distances != np.finfo(distances.dtype).max
+    node_ids = node_ids[mask]
+    if return_type != "path":
+        lengths = G._nodearrays_to_dict(node_ids, distances[mask])
+        if target is not None:
+            if target not in lengths:
+                raise nx.NetworkXNoPath(f"Node {target} not reachable from {source}")
+            lengths = lengths[target]
+    if return_type != "length":
+        if target is not None:
+            d = dict(zip(node_ids.tolist(), predecessors[mask].tolist()))
+            dst_index = target if G.key_to_id is None else G.key_to_id[target]
+            if dst_index not in d:
+                raise nx.NetworkXNoPath(f"Node {target} not reachable from {source}")
+            cur = dst_index
+            paths = [dst_index]
+            while cur != src_index:
+                cur = d[cur]
+                paths.append(cur)
+            if (id_to_key := G.id_to_key) is not None:
+                if reverse_path:
+                    paths = [id_to_key[cur] for cur in paths]
+                else:
+                    paths = [id_to_key[cur] for cur in reversed(paths)]
+            elif not reverse_path:
+                paths.reverse()
+        else:
+            groups = _groupby(predecessors[mask], node_ids)
+            if (id_to_key := G.id_to_key) is not None:
+                groups = {id_to_key[k]: v for k, v in groups.items() if k >= 0}
+            paths = {source: [source]}
+            preds = [source]
+            while preds:
+                pred = preds.pop()
+                pred_path = paths[pred]
+                nodes = G._nodearray_to_list(groups[pred])
+                if reverse_path:
+                    for node in nodes:
+                        paths[node] = [node, *pred_path]
+                else:
+                    for node in nodes:
+                        paths[node] = [*pred_path, node]
+                preds.extend(nodes & groups.keys())
+    if return_type == "path":
+        return paths
+    if return_type == "length":
+        return lengths
+    # return_type == "length-path"
+    return lengths, paths
diff --git a/python/nx-cugraph/nx_cugraph/interface.py b/python/nx-cugraph/nx_cugraph/interface.py
index d044ba6960d..0d893ac286b 100644
--- a/python/nx-cugraph/nx_cugraph/interface.py
+++ b/python/nx-cugraph/nx_cugraph/interface.py
@@ -67,6 +67,7 @@ def key(testpath):
         no_multigraph = "multigraphs not currently supported"
         louvain_different = "Louvain may be different due to RNG"
         no_string_dtype = "string edge values not currently supported"
+        sssp_path_different = "sssp may choose a different valid path"
 
         xfail = {
             # This is removed while strongly_connected_components() is not
@@ -77,6 +78,19 @@ def key(testpath):
             #     "test_strongly_connected.py:"
             #     "TestStronglyConnected.test_condensation_mapping_and_members"
             # ): "Strongly connected groups in different iteration order",
+            key(
+                "test_cycles.py:TestMinimumCycleBasis.test_unweighted_diamond"
+            ): sssp_path_different,
+            key(
+                "test_cycles.py:TestMinimumCycleBasis.test_weighted_diamond"
+            ): sssp_path_different,
+            key(
+                "test_cycles.py:TestMinimumCycleBasis.test_petersen_graph"
+            ): sssp_path_different,
+            key(
+                "test_cycles.py:TestMinimumCycleBasis."
+                "test_gh6787_and_edge_attribute_names"
+            ): sssp_path_different,
         }
 
         from packaging.version import parse
diff --git a/python/nx-cugraph/nx_cugraph/utils/misc.py b/python/nx-cugraph/nx_cugraph/utils/misc.py
index aa06d7fd29b..eab4b42c2cc 100644
--- a/python/nx-cugraph/nx_cugraph/utils/misc.py
+++ b/python/nx-cugraph/nx_cugraph/utils/misc.py
@@ -1,4 +1,4 @@
-# Copyright (c) 2023, NVIDIA CORPORATION.
+# Copyright (c) 2023-2024, NVIDIA CORPORATION.
 # Licensed 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
@@ -22,7 +22,9 @@
 import numpy as np
 
 if TYPE_CHECKING:
-    from ..typing import Dtype
+    import nx_cugraph as nxcg
+
+    from ..typing import Dtype, EdgeKey
 
 try:
     from itertools import pairwise  # Python >=3.10
@@ -190,10 +192,14 @@ def _get_int_dtype(
         raise ValueError("Value is too large to store as integer: {val}") from exc
 
 
-def _get_float_dtype(dtype: Dtype):
+def _get_float_dtype(
+    dtype: Dtype, *, graph: nxcg.Graph | None = None, weight: EdgeKey | None = None
+):
     """Promote dtype to float32 or float64 as appropriate."""
     if dtype is None:
-        return np.dtype(np.float32)
+        if graph is None or weight not in graph.edge_values:
+            return np.dtype(np.float32)
+        dtype = graph.edge_values[weight].dtype
     rv = np.promote_types(dtype, np.float32)
     if np.float32 != rv != np.float64:
         raise TypeError(
diff --git a/python/nx-cugraph/scripts/update_readme.py b/python/nx-cugraph/scripts/update_readme.py
old mode 100644
new mode 100755

From fda91fac9df7429febfa61251db1044aa1149fc1 Mon Sep 17 00:00:00 2001
From: Bradley Dice <bdice@bradleydice.com>
Date: Wed, 13 Mar 2024 10:22:13 -0500
Subject: [PATCH 4/5] Add upper bound to prevent usage of NumPy 2 (#4233)

NumPy 2 is expected to be released in the near future. For the RAPIDS 24.04 release, we will pin to `numpy>=1.23,<2.0a0`. This PR adds an upper bound to affected RAPIDS repositories.

xref: https://github.com/rapidsai/build-planning/issues/29

Authors:
  - Bradley Dice (https://github.com/bdice)

Approvers:
  - Rick Ratzel (https://github.com/rlratzel)
  - Ray Douglass (https://github.com/raydouglass)

URL: https://github.com/rapidsai/cugraph/pull/4233
---
 conda/environments/all_cuda-118_arch-x86_64.yaml | 2 +-
 conda/environments/all_cuda-122_arch-x86_64.yaml | 2 +-
 conda/recipes/cugraph-dgl/meta.yaml              | 2 +-
 conda/recipes/cugraph-pyg/meta.yaml              | 2 +-
 conda/recipes/cugraph-service/meta.yaml          | 2 +-
 dependencies.yaml                                | 2 +-
 python/cugraph-dgl/pyproject.toml                | 2 +-
 python/cugraph-pyg/pyproject.toml                | 2 +-
 python/cugraph-service/server/pyproject.toml     | 4 ++--
 python/cugraph/pyproject.toml                    | 4 ++--
 python/nx-cugraph/pyproject.toml                 | 2 +-
 python/pylibcugraph/pyproject.toml               | 2 +-
 12 files changed, 14 insertions(+), 14 deletions(-)

diff --git a/conda/environments/all_cuda-118_arch-x86_64.yaml b/conda/environments/all_cuda-118_arch-x86_64.yaml
index 6aed308c498..f0eff82e1ae 100644
--- a/conda/environments/all_cuda-118_arch-x86_64.yaml
+++ b/conda/environments/all_cuda-118_arch-x86_64.yaml
@@ -42,7 +42,7 @@ dependencies:
 - ninja
 - notebook>=0.5.0
 - numba>=0.57
-- numpy>=1.23
+- numpy>=1.23,<2.0a0
 - numpydoc
 - nvcc_linux-64=11.8
 - openmpi
diff --git a/conda/environments/all_cuda-122_arch-x86_64.yaml b/conda/environments/all_cuda-122_arch-x86_64.yaml
index 4a095058219..93972f40d8b 100644
--- a/conda/environments/all_cuda-122_arch-x86_64.yaml
+++ b/conda/environments/all_cuda-122_arch-x86_64.yaml
@@ -48,7 +48,7 @@ dependencies:
 - ninja
 - notebook>=0.5.0
 - numba>=0.57
-- numpy>=1.23
+- numpy>=1.23,<2.0a0
 - numpydoc
 - openmpi
 - packaging>=21
diff --git a/conda/recipes/cugraph-dgl/meta.yaml b/conda/recipes/cugraph-dgl/meta.yaml
index 09322a9c7d3..5e28e69a0d7 100644
--- a/conda/recipes/cugraph-dgl/meta.yaml
+++ b/conda/recipes/cugraph-dgl/meta.yaml
@@ -25,7 +25,7 @@ requirements:
     - cugraph ={{ version }}
     - dgl >=1.1.0.cu*
     - numba >=0.57
-    - numpy >=1.23
+    - numpy >=1.23,<2.0a0
     - pylibcugraphops ={{ minor_version }}
     - python
     - pytorch
diff --git a/conda/recipes/cugraph-pyg/meta.yaml b/conda/recipes/cugraph-pyg/meta.yaml
index 624f5753fd2..4ada5e31211 100644
--- a/conda/recipes/cugraph-pyg/meta.yaml
+++ b/conda/recipes/cugraph-pyg/meta.yaml
@@ -28,7 +28,7 @@ requirements:
   run:
     - rapids-dask-dependency ={{ minor_version }}
     - numba >=0.57
-    - numpy >=1.23
+    - numpy >=1.23,<2.0a0
     - python
     - pytorch >=2.0
     - cupy >=12.0.0
diff --git a/conda/recipes/cugraph-service/meta.yaml b/conda/recipes/cugraph-service/meta.yaml
index c04c1a7c7fa..8698d4f6985 100644
--- a/conda/recipes/cugraph-service/meta.yaml
+++ b/conda/recipes/cugraph-service/meta.yaml
@@ -60,7 +60,7 @@ outputs:
         - dask-cuda ={{ minor_version }}
         - dask-cudf ={{ minor_version }}
         - numba >=0.57
-        - numpy >=1.23
+        - numpy >=1.23,<2.0a0
         - python
         - rapids-dask-dependency ={{ minor_version }}
         - thriftpy2 >=0.4.15
diff --git a/dependencies.yaml b/dependencies.yaml
index e6cf6c9e93c..d8be5352c7d 100644
--- a/dependencies.yaml
+++ b/dependencies.yaml
@@ -449,7 +449,7 @@ dependencies:
           - &dask rapids-dask-dependency==24.4.*
           - &dask_cuda dask-cuda==24.4.*
           - &numba numba>=0.57
-          - &numpy numpy>=1.23
+          - &numpy numpy>=1.23,<2.0a0
           - &ucx_py ucx-py==0.37.*
       - output_types: conda
         packages:
diff --git a/python/cugraph-dgl/pyproject.toml b/python/cugraph-dgl/pyproject.toml
index c6f76325761..f17292c5e70 100644
--- a/python/cugraph-dgl/pyproject.toml
+++ b/python/cugraph-dgl/pyproject.toml
@@ -25,7 +25,7 @@ classifiers = [
 dependencies = [
     "cugraph==24.4.*",
     "numba>=0.57",
-    "numpy>=1.23",
+    "numpy>=1.23,<2.0a0",
     "pylibcugraphops==24.4.*",
 ] # This list was generated by `rapids-dependency-file-generator`. To make changes, edit ../../dependencies.yaml and run `rapids-dependency-file-generator`.
 
diff --git a/python/cugraph-pyg/pyproject.toml b/python/cugraph-pyg/pyproject.toml
index cbee5ed4b58..150ecbf506b 100644
--- a/python/cugraph-pyg/pyproject.toml
+++ b/python/cugraph-pyg/pyproject.toml
@@ -29,7 +29,7 @@ classifiers = [
 dependencies = [
     "cugraph==24.4.*",
     "numba>=0.57",
-    "numpy>=1.23",
+    "numpy>=1.23,<2.0a0",
     "pylibcugraphops==24.4.*",
 ] # This list was generated by `rapids-dependency-file-generator`. To make changes, edit ../../dependencies.yaml and run `rapids-dependency-file-generator`.
 
diff --git a/python/cugraph-service/server/pyproject.toml b/python/cugraph-service/server/pyproject.toml
index a32b18a9551..d6cf48432cb 100644
--- a/python/cugraph-service/server/pyproject.toml
+++ b/python/cugraph-service/server/pyproject.toml
@@ -26,7 +26,7 @@ dependencies = [
     "dask-cuda==24.4.*",
     "dask-cudf==24.4.*",
     "numba>=0.57",
-    "numpy>=1.23",
+    "numpy>=1.23,<2.0a0",
     "rapids-dask-dependency==24.4.*",
     "rmm==24.4.*",
     "thriftpy2",
@@ -46,7 +46,7 @@ cugraph-service-server = "cugraph_service_server.__main__:main"
 [project.optional-dependencies]
 test = [
     "networkx>=2.5.1",
-    "numpy>=1.23",
+    "numpy>=1.23,<2.0a0",
     "pandas",
     "pytest",
     "pytest-benchmark",
diff --git a/python/cugraph/pyproject.toml b/python/cugraph/pyproject.toml
index 113c316ccbf..a6d3d841298 100644
--- a/python/cugraph/pyproject.toml
+++ b/python/cugraph/pyproject.toml
@@ -35,7 +35,7 @@ dependencies = [
     "dask-cudf==24.4.*",
     "fsspec[http]>=0.6.0",
     "numba>=0.57",
-    "numpy>=1.23",
+    "numpy>=1.23,<2.0a0",
     "pylibcugraph==24.4.*",
     "raft-dask==24.4.*",
     "rapids-dask-dependency==24.4.*",
@@ -53,7 +53,7 @@ classifiers = [
 [project.optional-dependencies]
 test = [
     "networkx>=2.5.1",
-    "numpy>=1.23",
+    "numpy>=1.23,<2.0a0",
     "pandas",
     "pytest",
     "pytest-benchmark",
diff --git a/python/nx-cugraph/pyproject.toml b/python/nx-cugraph/pyproject.toml
index 07ec0eab264..dbdc8dd19e1 100644
--- a/python/nx-cugraph/pyproject.toml
+++ b/python/nx-cugraph/pyproject.toml
@@ -33,7 +33,7 @@ classifiers = [
 dependencies = [
     "cupy-cuda11x>=12.0.0",
     "networkx>=3.0",
-    "numpy>=1.23",
+    "numpy>=1.23,<2.0a0",
     "pylibcugraph==24.4.*",
 ] # This list was generated by `rapids-dependency-file-generator`. To make changes, edit ../../dependencies.yaml and run `rapids-dependency-file-generator`.
 
diff --git a/python/pylibcugraph/pyproject.toml b/python/pylibcugraph/pyproject.toml
index eb7323d19e5..d5f568a7a90 100644
--- a/python/pylibcugraph/pyproject.toml
+++ b/python/pylibcugraph/pyproject.toml
@@ -42,7 +42,7 @@ classifiers = [
 [project.optional-dependencies]
 test = [
     "cudf==24.4.*",
-    "numpy>=1.23",
+    "numpy>=1.23,<2.0a0",
     "pandas",
     "pytest",
     "pytest-benchmark",

From 3086f83a3331c08ed6834dbd8e02b91fab8fff26 Mon Sep 17 00:00:00 2001
From: Kyle Edwards <kyedwards@nvidia.com>
Date: Wed, 13 Mar 2024 15:20:37 -0400
Subject: [PATCH 5/5] Remove hard-coding of RAPIDS version where possible
 (#4217)

* Read `VERSION` file in CMake
* Read `cugraph.__version__` in docs build
* Read `VERSION` file in shell scripts
* Use environment variables in Doxyfile
* Remove updates from `ci/update-version.sh`

Issue: https://github.com/rapidsai/build-planning/issues/15

Authors:
  - Kyle Edwards (https://github.com/KyleFromNVIDIA)

Approvers:
  - Don Acosta (https://github.com/acostadon)
  - Chuck Hastings (https://github.com/ChuckHastings)
  - Brad Rees (https://github.com/BradReesWork)
  - Jake Awe (https://github.com/AyodeAwe)
  - https://github.com/jakirkham

URL: https://github.com/rapidsai/cugraph/pull/4217
---
 build.sh                                      |  2 +-
 ci/build_docs.sh                              |  4 ++-
 ci/build_python.sh                            |  3 +-
 ci/build_wheel.sh                             |  6 +++-
 ci/release/update-version.sh                  | 31 -----------------
 cpp/CMakeLists.txt                            |  6 ++--
 cpp/doxygen/Doxyfile                          |  2 +-
 cpp/libcugraph_etl/CMakeLists.txt             |  4 +--
 docs/cugraph/source/conf.py                   |  9 +++--
 fetch_rapids.cmake                            | 19 -----------
 python/cugraph/CMakeLists.txt                 |  8 ++---
 .../{nx_cugraph => _nx_cugraph}/VERSION       |  0
 python/nx-cugraph/_nx_cugraph/__init__.py     | 11 +++---
 .../{nx_cugraph => _nx_cugraph}/_version.py   |  2 +-
 python/nx-cugraph/nx_cugraph/__init__.py      |  4 +--
 python/nx-cugraph/pyproject.toml              |  2 +-
 python/pylibcugraph/CMakeLists.txt            |  8 ++---
 rapids_config.cmake                           | 34 +++++++++++++++++++
 18 files changed, 75 insertions(+), 80 deletions(-)
 delete mode 100644 fetch_rapids.cmake
 rename python/nx-cugraph/{nx_cugraph => _nx_cugraph}/VERSION (100%)
 rename python/nx-cugraph/{nx_cugraph => _nx_cugraph}/_version.py (91%)
 create mode 100644 rapids_config.cmake

diff --git a/build.sh b/build.sh
index 5cfd2b5af1c..bd716943216 100755
--- a/build.sh
+++ b/build.sh
@@ -18,7 +18,7 @@ ARGS=$*
 # script, and that this script resides in the repo dir!
 REPODIR=$(cd $(dirname $0); pwd)
 
-RAPIDS_VERSION=24.04
+RAPIDS_VERSION="$(sed -E -e 's/^([0-9]{2})\.([0-9]{2})\.([0-9]{2}).*$/\1.\2/' VERSION)"
 
 # Valid args to this script (all possible targets and options) - only one per line
 VALIDARGS="
diff --git a/ci/build_docs.sh b/ci/build_docs.sh
index 298a8b68791..8e7cfc2c4b8 100755
--- a/ci/build_docs.sh
+++ b/ci/build_docs.sh
@@ -39,7 +39,9 @@ rapids-mamba-retry install \
 rapids-logger "Install cugraph-dgl"
 rapids-mamba-retry install "${PYTHON_CHANNEL}/linux-64/cugraph-dgl-*.tar.bz2"
 
-export RAPIDS_VERSION_NUMBER="24.04"
+export RAPIDS_VERSION="$(rapids-version)"
+export RAPIDS_VERSION_MAJOR_MINOR="$(rapids-version-major-minor)"
+export RAPIDS_VERSION_NUMBER="$RAPIDS_VERSION_MAJOR_MINOR"
 export RAPIDS_DOCS_DIR="$(mktemp -d)"
 
 for PROJECT in libcugraphops libwholegraph; do
diff --git a/ci/build_python.sh b/ci/build_python.sh
index 07a4f59396b..62154fdeced 100755
--- a/ci/build_python.sh
+++ b/ci/build_python.sh
@@ -23,10 +23,11 @@ echo "${version}" > VERSION
 rapids-logger "Begin py build"
 
 package_dir="python"
-for package_name in pylibcugraph cugraph nx-cugraph cugraph-pyg cugraph-dgl; do
+for package_name in pylibcugraph cugraph cugraph-pyg cugraph-dgl; do
   underscore_package_name=$(echo "${package_name}" | tr "-" "_")
   sed -i "/^__git_commit__/ s/= .*/= \"${git_commit}\"/g" "${package_dir}/${package_name}/${underscore_package_name}/_version.py"
 done
+sed -i "/^__git_commit__/ s/= .*/= \"${git_commit}\"/g" "${package_dir}/nx-cugraph/_nx_cugraph/_version.py"
 
 # TODO: Remove `--no-test` flags once importing on a CPU
 # node works correctly
diff --git a/ci/build_wheel.sh b/ci/build_wheel.sh
index 9de1750de81..587c5fb38e7 100755
--- a/ci/build_wheel.sh
+++ b/ci/build_wheel.sh
@@ -20,8 +20,12 @@ RAPIDS_PY_CUDA_SUFFIX="$(rapids-wheel-ctk-name-gen ${RAPIDS_CUDA_VERSION})"
 PACKAGE_CUDA_SUFFIX="-${RAPIDS_PY_CUDA_SUFFIX}"
 
 # Patch project metadata files to include the CUDA version suffix and version override.
+version_package_name="$underscore_package_name"
+if [[ "${version_package_name}" = "nx_cugraph" ]]; then
+    version_package_name="_nx_cugraph"
+fi
 pyproject_file="${package_dir}/pyproject.toml"
-version_file="${package_dir}/${underscore_package_name}/_version.py"
+version_file="${package_dir}/${version_package_name}/_version.py"
 
 sed -i "s/name = \"${package_name}\"/name = \"${package_name}${PACKAGE_CUDA_SUFFIX}\"/g" ${pyproject_file}
 echo "${version}" > VERSION
diff --git a/ci/release/update-version.sh b/ci/release/update-version.sh
index 9e284f49b5b..ce2c14a01fc 100755
--- a/ci/release/update-version.sh
+++ b/ci/release/update-version.sh
@@ -39,33 +39,11 @@ function sed_runner() {
     sed -i.bak ''"$1"'' $2 && rm -f ${2}.bak
 }
 
-# rapids-cmake version
-sed_runner 's/'"branch-.*\/RAPIDS.cmake"'/'"branch-${NEXT_SHORT_TAG}\/RAPIDS.cmake"'/g' fetch_rapids.cmake
-
-# CMakeLists update
-sed_runner 's/'"CUGRAPH VERSION .* LANGUAGES C CXX CUDA)"'/'"CUGRAPH VERSION ${NEXT_FULL_TAG} LANGUAGES C CXX CUDA)"'/g' cpp/CMakeLists.txt
-sed_runner 's|'"branch-.*/RAPIDS.cmake"'|'"branch-${NEXT_SHORT_TAG}/RAPIDS.cmake"'|g' cpp/CMakeLists.txt
-sed_runner 's/'"CUGRAPH_ETL VERSION .* LANGUAGES C CXX CUDA)"'/'"CUGRAPH_ETL VERSION ${NEXT_FULL_TAG} LANGUAGES C CXX CUDA)"'/g' cpp/libcugraph_etl/CMakeLists.txt
-sed_runner 's|'"branch-.*/RAPIDS.cmake"'|'"branch-${NEXT_SHORT_TAG}/RAPIDS.cmake"'|g' cpp/libcugraph_etl/CMakeLists.txt
-sed_runner "s/set(pylibcugraph_version .*)/set(pylibcugraph_version ${NEXT_FULL_TAG})/g" python/pylibcugraph/CMakeLists.txt
-sed_runner "s/set(cugraph_version .*)/set(cugraph_version ${NEXT_FULL_TAG})/g" python/cugraph/CMakeLists.txt
-
-# RTD update
-sed_runner 's/version = .*/version = '"'${NEXT_SHORT_TAG}'"'/g' docs/cugraph/source/conf.py
-sed_runner 's/release = .*/release = '"'${NEXT_FULL_TAG}'"'/g' docs/cugraph/source/conf.py
-
-
-# build.sh script
-sed_runner 's/RAPIDS_VERSION=.*/RAPIDS_VERSION='${NEXT_SHORT_TAG}'/g' build.sh
-
 # Centralized version file update
 # NOTE: Any script that runs in CI will need to use gha-tool `rapids-generate-version`
 # and echo it to `VERSION` file to get an alpha spec of the current version
 echo "${NEXT_FULL_TAG}" > VERSION
 
-# Wheel testing script
-sed_runner "s/branch-.*/branch-${NEXT_SHORT_TAG}/g" ci/test_wheel_cugraph.sh
-
 # Need to distutils-normalize the original version
 NEXT_SHORT_TAG_PEP440=$(python -c "from setuptools.extern import packaging; print(packaging.version.Version('${NEXT_SHORT_TAG}'))")
 
@@ -105,19 +83,11 @@ for DEP in "${DEPENDENCIES[@]}"; do
   done
 done
 
-# Doxyfile update
-sed_runner "s|PROJECT_NUMBER[[:space:]]*=.*|PROJECT_NUMBER=${NEXT_SHORT_TAG}|" cpp/doxygen/Doxyfile
-
 # ucx-py version
 sed_runner "/^ucx_py_version:$/ {n;s/.*/  - \"${NEXT_UCX_PY_VERSION}.*\"/}" conda/recipes/cugraph/conda_build_config.yaml
 sed_runner "/^ucx_py_version:$/ {n;s/.*/  - \"${NEXT_UCX_PY_VERSION}.*\"/}" conda/recipes/cugraph-service/conda_build_config.yaml
 sed_runner "/^ucx_py_version:$/ {n;s/.*/  - \"${NEXT_UCX_PY_VERSION}.*\"/}" conda/recipes/pylibcugraph/conda_build_config.yaml
 
-# nx-cugraph NetworkX entry-point meta-data
-sed_runner "s@branch-[0-9][0-9].[0-9][0-9]@branch-${NEXT_SHORT_TAG}@g" python/nx-cugraph/_nx_cugraph/__init__.py
-# FIXME: can this use the standard VERSION file and update mechanism?
-sed_runner "s/__version__ = .*/__version__ = \"${NEXT_FULL_TAG}\"/g" python/nx-cugraph/_nx_cugraph/__init__.py
-
 # CI files
 for FILE in .github/workflows/*.yaml; do
   sed_runner "/shared-workflows/ s/@.*/@branch-${NEXT_SHORT_TAG}/g" "${FILE}"
@@ -126,7 +96,6 @@ for FILE in .github/workflows/*.yaml; do
   # Wheel builds install dask-cuda from source, update its branch
   sed_runner "s/dask-cuda.git@branch-[0-9][0-9].[0-9][0-9]/dask-cuda.git@branch-${NEXT_SHORT_TAG}/g" "${FILE}"
 done
-sed_runner "s/RAPIDS_VERSION_NUMBER=\".*/RAPIDS_VERSION_NUMBER=\"${NEXT_SHORT_TAG}\"/g" ci/build_docs.sh
 
 sed_runner "s/branch-.*/branch-${NEXT_SHORT_TAG}/g" python/nx-cugraph/README.md
 
diff --git a/cpp/CMakeLists.txt b/cpp/CMakeLists.txt
index 88908ef70ce..6070621134d 100644
--- a/cpp/CMakeLists.txt
+++ b/cpp/CMakeLists.txt
@@ -16,7 +16,7 @@
 
 cmake_minimum_required(VERSION 3.26.4 FATAL_ERROR)
 
-include(../fetch_rapids.cmake)
+include(../rapids_config.cmake)
 include(rapids-cmake)
 include(rapids-cpm)
 include(rapids-cuda)
@@ -25,7 +25,7 @@ include(rapids-find)
 
 rapids_cuda_init_architectures(CUGRAPH)
 
-project(CUGRAPH VERSION 24.04.00 LANGUAGES C CXX CUDA)
+project(CUGRAPH VERSION "${RAPIDS_VERSION}" LANGUAGES C CXX CUDA)
 
 if(CMAKE_CUDA_COMPILER_ID STREQUAL "NVIDIA" AND
    CMAKE_CUDA_COMPILER_VERSION VERSION_LESS 11.0)
@@ -590,7 +590,7 @@ find_package(Doxygen 1.8.11)
 if(Doxygen_FOUND)
     add_custom_command(OUTPUT CUGRAPH_DOXYGEN
                        WORKING_DIRECTORY ${CMAKE_CURRENT_SOURCE_DIR}/doxygen
-                       COMMAND doxygen Doxyfile
+                       COMMAND ${CMAKE_COMMAND} -E env "RAPIDS_VERSION_MAJOR_MINOR=${RAPIDS_VERSION_MAJOR_MINOR}" doxygen Doxyfile
                        VERBATIM)
 
     add_custom_target(docs_cugraph DEPENDS CUGRAPH_DOXYGEN)
diff --git a/cpp/doxygen/Doxyfile b/cpp/doxygen/Doxyfile
index d4652a79f91..5b0f2ec0a5e 100644
--- a/cpp/doxygen/Doxyfile
+++ b/cpp/doxygen/Doxyfile
@@ -48,7 +48,7 @@ PROJECT_NAME           = libcugraph
 # could be handy for archiving the generated documentation or if some version
 # control system is used.
 
-PROJECT_NUMBER=24.04
+PROJECT_NUMBER=$(RAPIDS_VERSION_MAJOR_MINOR)
 
 # Using the PROJECT_BRIEF tag one can provide an optional one line description
 # for a project that appears at the top of each page and should give viewer a
diff --git a/cpp/libcugraph_etl/CMakeLists.txt b/cpp/libcugraph_etl/CMakeLists.txt
index c1b526fb121..6b258afdf96 100644
--- a/cpp/libcugraph_etl/CMakeLists.txt
+++ b/cpp/libcugraph_etl/CMakeLists.txt
@@ -15,7 +15,7 @@
 #=============================================================================
 
 cmake_minimum_required(VERSION 3.26.4 FATAL_ERROR)
-include(../../fetch_rapids.cmake)
+include(../../rapids_config.cmake)
 
 include(rapids-cmake)
 include(rapids-cpm)
@@ -25,7 +25,7 @@ include(rapids-find)
 
 rapids_cuda_init_architectures(CUGRAPH_ETL)
 
-project(CUGRAPH_ETL VERSION 24.04.00 LANGUAGES C CXX CUDA)
+project(CUGRAPH_ETL VERSION "${RAPIDS_VERSION}" LANGUAGES C CXX CUDA)
 
 if(CMAKE_CUDA_COMPILER_ID STREQUAL "NVIDIA" AND
    CMAKE_CUDA_COMPILER_VERSION VERSION_LESS 11.0)
diff --git a/docs/cugraph/source/conf.py b/docs/cugraph/source/conf.py
index 719f94d4838..952b962aca2 100644
--- a/docs/cugraph/source/conf.py
+++ b/docs/cugraph/source/conf.py
@@ -19,6 +19,10 @@
 import os
 import sys
 
+from packaging.version import Version
+
+import cugraph
+
 # If extensions (or modules to document with autodoc) are in another
 # directory, add these directories to sys.path here. If the directory
 # is relative to the documentation root, use os.path.abspath to make it
@@ -76,10 +80,11 @@
 # |version| and |release|, also used in various other places throughout the
 # built documents.
 #
+CUGRAPH_VERSION = Version(cugraph.__version__)
 # The short X.Y version.
-version = '24.04'
+version = f"{CUGRAPH_VERSION.major:02}.{CUGRAPH_VERSION.minor:02}"
 # The full version, including alpha/beta/rc tags.
-release = '24.04.00'
+release = f"{CUGRAPH_VERSION.major:02}.{CUGRAPH_VERSION.minor:02}.{CUGRAPH_VERSION.micro:02}"
 
 # The language for content autogenerated by Sphinx. Refer to documentation
 # for a list of supported languages.
diff --git a/fetch_rapids.cmake b/fetch_rapids.cmake
deleted file mode 100644
index 596908d01cd..00000000000
--- a/fetch_rapids.cmake
+++ /dev/null
@@ -1,19 +0,0 @@
-# =============================================================================
-# Copyright (c) 2022-2024, NVIDIA CORPORATION.
-#
-# Licensed 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.
-# =============================================================================
-if(NOT EXISTS ${CMAKE_CURRENT_BINARY_DIR}/CUGRAPH_RAPIDS.cmake)
-  file(DOWNLOAD https://raw.githubusercontent.com/rapidsai/rapids-cmake/branch-24.04/RAPIDS.cmake
-       ${CMAKE_CURRENT_BINARY_DIR}/CUGRAPH_RAPIDS.cmake
-  )
-endif()
-include(${CMAKE_CURRENT_BINARY_DIR}/CUGRAPH_RAPIDS.cmake)
diff --git a/python/cugraph/CMakeLists.txt b/python/cugraph/CMakeLists.txt
index 79510f9bf8d..dfccf02d042 100644
--- a/python/cugraph/CMakeLists.txt
+++ b/python/cugraph/CMakeLists.txt
@@ -14,9 +14,7 @@
 
 cmake_minimum_required(VERSION 3.26.4 FATAL_ERROR)
 
-set(cugraph_version 24.04.00)
-
-include(../../fetch_rapids.cmake)
+include(../../rapids_config.cmake)
 
 # We always need CUDA for cuML because the raft dependency brings in a
 # header-only cuco dependency that enables CUDA unconditionally.
@@ -25,7 +23,7 @@ rapids_cuda_init_architectures(cugraph-python)
 
 project(
   cugraph-python
-  VERSION ${cugraph_version}
+  VERSION "${RAPIDS_VERSION}"
   LANGUAGES CXX CUDA
 )
 
@@ -43,7 +41,7 @@ endif()
 
 # If the user requested it,  we attempt to find CUGRAPH.
 if(FIND_CUGRAPH_CPP)
-  find_package(cugraph ${cugraph_version} REQUIRED)
+  find_package(cugraph "${RAPIDS_VERSION}" REQUIRED)
 else()
   set(cugraph_FOUND OFF)
 endif()
diff --git a/python/nx-cugraph/nx_cugraph/VERSION b/python/nx-cugraph/_nx_cugraph/VERSION
similarity index 100%
rename from python/nx-cugraph/nx_cugraph/VERSION
rename to python/nx-cugraph/_nx_cugraph/VERSION
diff --git a/python/nx-cugraph/_nx_cugraph/__init__.py b/python/nx-cugraph/_nx_cugraph/__init__.py
index bc7f63fcd49..8c6a6504675 100644
--- a/python/nx-cugraph/_nx_cugraph/__init__.py
+++ b/python/nx-cugraph/_nx_cugraph/__init__.py
@@ -23,12 +23,18 @@
 $ python _nx_cugraph/__init__.py
 """
 
+from packaging.version import Version
+
+from _nx_cugraph._version import __version__
+
+_nx_cugraph_version = Version(__version__)
+
 # Entries between BEGIN and END are automatically generated
 _info = {
     "backend_name": "cugraph",
     "project": "nx-cugraph",
     "package": "nx_cugraph",
-    "url": "https://github.com/rapidsai/cugraph/tree/branch-24.04/python/nx-cugraph",
+    "url": f"https://github.com/rapidsai/cugraph/tree/branch-{_nx_cugraph_version.major:02}.{_nx_cugraph_version.minor:02}/python/nx-cugraph",
     "short_summary": "GPU-accelerated backend.",
     # "description": "TODO",
     "functions": {
@@ -252,9 +258,6 @@ def get_info():
     return d
 
 
-# FIXME: can this use the standard VERSION file and update mechanism?
-__version__ = "24.04.00"
-
 if __name__ == "__main__":
     from pathlib import Path
 
diff --git a/python/nx-cugraph/nx_cugraph/_version.py b/python/nx-cugraph/_nx_cugraph/_version.py
similarity index 91%
rename from python/nx-cugraph/nx_cugraph/_version.py
rename to python/nx-cugraph/_nx_cugraph/_version.py
index a528a3bfe1b..dc2d2a3a3c0 100644
--- a/python/nx-cugraph/nx_cugraph/_version.py
+++ b/python/nx-cugraph/_nx_cugraph/_version.py
@@ -19,6 +19,6 @@
 # to update version info (including commit hashes) without modifying
 # source files.
 __version__ = (
-    importlib.resources.files("nx_cugraph").joinpath("VERSION").read_text().strip()
+    importlib.resources.files("_nx_cugraph").joinpath("VERSION").read_text().strip()
 )
 __git_commit__ = ""
diff --git a/python/nx-cugraph/nx_cugraph/__init__.py b/python/nx-cugraph/nx_cugraph/__init__.py
index 3a8f0996e9c..2c54da87898 100644
--- a/python/nx-cugraph/nx_cugraph/__init__.py
+++ b/python/nx-cugraph/nx_cugraph/__init__.py
@@ -1,4 +1,4 @@
-# Copyright (c) 2023, NVIDIA CORPORATION.
+# Copyright (c) 2023-2024, NVIDIA CORPORATION.
 # Licensed 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
@@ -29,4 +29,4 @@
 from . import algorithms
 from .algorithms import *
 
-from nx_cugraph._version import __git_commit__, __version__
+from _nx_cugraph._version import __git_commit__, __version__
diff --git a/python/nx-cugraph/pyproject.toml b/python/nx-cugraph/pyproject.toml
index dbdc8dd19e1..2ff53c1a3f6 100644
--- a/python/nx-cugraph/pyproject.toml
+++ b/python/nx-cugraph/pyproject.toml
@@ -71,7 +71,7 @@ cugraph = "_nx_cugraph:get_info"
 license-files = ["LICENSE"]
 
 [tool.setuptools.dynamic]
-version = {file = "nx_cugraph/VERSION"}
+version = {file = "_nx_cugraph/VERSION"}
 
 [tool.setuptools.packages.find]
 include = [
diff --git a/python/pylibcugraph/CMakeLists.txt b/python/pylibcugraph/CMakeLists.txt
index cc76ab5ba2b..f43b7db1279 100644
--- a/python/pylibcugraph/CMakeLists.txt
+++ b/python/pylibcugraph/CMakeLists.txt
@@ -14,9 +14,7 @@
 
 cmake_minimum_required(VERSION 3.26.4 FATAL_ERROR)
 
-set(pylibcugraph_version 24.04.00)
-
-include(../../fetch_rapids.cmake)
+include(../../rapids_config.cmake)
 
 # We always need CUDA for cuML because the raft dependency brings in a
 # header-only cuco dependency that enables CUDA unconditionally.
@@ -25,7 +23,7 @@ rapids_cuda_init_architectures(pylibcugraph-python)
 
 project(
   pylibcugraph-python
-  VERSION ${pylibcugraph_version}
+  VERSION "${RAPIDS_VERSION}"
   LANGUAGES CXX CUDA
 )
 
@@ -43,7 +41,7 @@ endif()
 
 # If the user requested it we attempt to find CUGRAPH.
 if(FIND_CUGRAPH_CPP)
-  find_package(cugraph ${pylibcugraph_version} REQUIRED)
+  find_package(cugraph "${RAPIDS_VERSION}" REQUIRED)
 else()
   set(cugraph_FOUND OFF)
 endif()
diff --git a/rapids_config.cmake b/rapids_config.cmake
new file mode 100644
index 00000000000..50b1054b7b9
--- /dev/null
+++ b/rapids_config.cmake
@@ -0,0 +1,34 @@
+# =============================================================================
+# Copyright (c) 2018-2024, NVIDIA CORPORATION.
+#
+# Licensed 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.
+# =============================================================================
+file(READ "${CMAKE_CURRENT_LIST_DIR}/VERSION" _rapids_version)
+if(_rapids_version MATCHES [[^([0-9][0-9])\.([0-9][0-9])\.([0-9][0-9])]])
+  set(RAPIDS_VERSION_MAJOR "${CMAKE_MATCH_1}")
+  set(RAPIDS_VERSION_MINOR "${CMAKE_MATCH_2}")
+  set(RAPIDS_VERSION_PATCH "${CMAKE_MATCH_3}")
+  set(RAPIDS_VERSION_MAJOR_MINOR "${RAPIDS_VERSION_MAJOR}.${RAPIDS_VERSION_MINOR}")
+  set(RAPIDS_VERSION "${RAPIDS_VERSION_MAJOR}.${RAPIDS_VERSION_MINOR}.${RAPIDS_VERSION_PATCH}")
+else()
+  string(REPLACE "\n" "\n  " _rapids_version_formatted "  ${_rapids_version}")
+  message(
+    FATAL_ERROR
+      "Could not determine RAPIDS version. Contents of VERSION file:\n${_rapids_version_formatted}")
+endif()
+
+if(NOT EXISTS "${CMAKE_CURRENT_BINARY_DIR}/CUGRAPH_RAPIDS-${RAPIDS_VERSION_MAJOR_MINOR}.cmake")
+  file(
+    DOWNLOAD
+    "https://raw.githubusercontent.com/rapidsai/rapids-cmake/branch-${RAPIDS_VERSION_MAJOR_MINOR}/RAPIDS.cmake"
+    "${CMAKE_CURRENT_BINARY_DIR}/CUGRAPH_RAPIDS-${RAPIDS_VERSION_MAJOR_MINOR}.cmake")
+endif()
+include("${CMAKE_CURRENT_BINARY_DIR}/CUGRAPH_RAPIDS-${RAPIDS_VERSION_MAJOR_MINOR}.cmake")