diff --git a/cpp/CMakeLists.txt b/cpp/CMakeLists.txt index 32a753c9f40..674e1292579 100644 --- a/cpp/CMakeLists.txt +++ b/cpp/CMakeLists.txt @@ -368,8 +368,12 @@ add_library( src/filling/repeat.cu src/filling/sequence.cu src/groupby/groupby.cu + src/groupby/hash/compute_groupby.cu + src/groupby/hash/compute_single_pass_aggs.cu src/groupby/hash/flatten_single_pass_aggs.cpp src/groupby/hash/groupby.cu + src/groupby/hash/hash_compound_agg_finalizer.cu + src/groupby/hash/sparse_to_dense_results.cu src/groupby/sort/aggregate.cpp src/groupby/sort/group_argmax.cu src/groupby/sort/group_argmin.cu diff --git a/cpp/src/groupby/hash/compute_groupby.cu b/cpp/src/groupby/hash/compute_groupby.cu new file mode 100644 index 00000000000..37dd2657663 --- /dev/null +++ b/cpp/src/groupby/hash/compute_groupby.cu @@ -0,0 +1,171 @@ +/* + * 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 "compute_single_pass_aggs.hpp" +#include "helpers.cuh" +#include "sparse_to_dense_results.hpp" + +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include + +#include + +#include +#include + +namespace cudf::groupby::detail::hash { +/** + * @brief Computes and returns a device vector containing all populated keys in + * `map`. + */ +template +rmm::device_uvector extract_populated_keys(SetType const& key_set, + size_type num_keys, + rmm::cuda_stream_view stream) +{ + rmm::device_uvector populated_keys(num_keys, stream); + auto const keys_end = key_set.retrieve_all(populated_keys.begin(), stream.value()); + + populated_keys.resize(std::distance(populated_keys.begin(), keys_end), stream); + return populated_keys; +} + +/** + * @brief Computes groupby using hash table. + * + * First, we create a hash table that stores the indices of unique rows in + * `keys`. The upper limit on the number of values in this map is the number + * of rows in `keys`. + * + * To store the results of aggregations, we create temporary sparse columns + * which have the same size as input value columns. Using the hash map, we + * determine the location within the sparse column to write the result of the + * aggregation into. + * + * The sparse column results of all aggregations are stored into the cache + * `sparse_results`. This enables the use of previously calculated results in + * other aggregations. + * + * All the aggregations which can be computed in a single pass are computed + * first, in a combined kernel. Then using these results, aggregations that + * require multiple passes, will be computed. + * + * Finally, using the hash map, we generate a vector of indices of populated + * values in sparse result columns. Then, for each aggregation originally + * requested in `requests`, we gather sparse results into a column of dense + * results using the aforementioned index vector. Dense results are stored into + * the in/out parameter `cache`. + */ +template +std::unique_ptr compute_groupby(table_view const& keys, + host_span requests, + bool skip_rows_with_nulls, + Equal const& d_row_equal, + row_hash_t const& d_row_hash, + cudf::detail::result_cache* cache, + rmm::cuda_stream_view stream, + rmm::device_async_resource_ref mr) +{ + // convert to int64_t to avoid potential overflow with large `keys` + auto const num_keys = static_cast(keys.num_rows()); + + // Cache of sparse results where the location of aggregate value in each + // column is indexed by the hash set + cudf::detail::result_cache sparse_results(requests.size()); + + auto const set = cuco::static_set{ + num_keys, + cudf::detail::CUCO_DESIRED_LOAD_FACTOR, // 50% load factor + cuco::empty_key{cudf::detail::CUDF_SIZE_TYPE_SENTINEL}, + d_row_equal, + probing_scheme_t{d_row_hash}, + cuco::thread_scope_device, + cuco::storage{}, + cudf::detail::cuco_allocator{rmm::mr::polymorphic_allocator{}, stream}, + stream.value()}; + + auto row_bitmask = + skip_rows_with_nulls + ? cudf::bitmask_and(keys, stream, cudf::get_current_device_resource_ref()).first + : rmm::device_buffer{}; + + // Compute all single pass aggs first + compute_single_pass_aggs(num_keys, + skip_rows_with_nulls, + static_cast(row_bitmask.data()), + set.ref(cuco::insert_and_find), + requests, + &sparse_results, + stream); + + // Extract the populated indices from the hash set and create a gather map. + // Gathering using this map from sparse results will give dense results. + auto gather_map = extract_populated_keys(set, keys.num_rows(), stream); + + // Compact all results from sparse_results and insert into cache + sparse_to_dense_results(requests, + &sparse_results, + cache, + gather_map, + set.ref(cuco::find), + static_cast(row_bitmask.data()), + stream, + mr); + + return cudf::detail::gather(keys, + gather_map, + out_of_bounds_policy::DONT_CHECK, + cudf::detail::negative_index_policy::NOT_ALLOWED, + stream, + mr); +} + +template rmm::device_uvector extract_populated_keys( + global_set_t const& key_set, size_type num_keys, rmm::cuda_stream_view stream); + +template rmm::device_uvector extract_populated_keys( + nullable_global_set_t const& key_set, size_type num_keys, rmm::cuda_stream_view stream); + +template std::unique_ptr
compute_groupby( + table_view const& keys, + host_span requests, + bool skip_rows_with_nulls, + row_comparator_t const& d_row_equal, + row_hash_t const& d_row_hash, + cudf::detail::result_cache* cache, + rmm::cuda_stream_view stream, + rmm::device_async_resource_ref mr); + +template std::unique_ptr
compute_groupby( + table_view const& keys, + host_span requests, + bool skip_rows_with_nulls, + nullable_row_comparator_t const& d_row_equal, + row_hash_t const& d_row_hash, + cudf::detail::result_cache* cache, + rmm::cuda_stream_view stream, + rmm::device_async_resource_ref mr); +} // namespace cudf::groupby::detail::hash diff --git a/cpp/src/groupby/hash/compute_groupby.hpp b/cpp/src/groupby/hash/compute_groupby.hpp new file mode 100644 index 00000000000..a3def9abdb1 --- /dev/null +++ b/cpp/src/groupby/hash/compute_groupby.hpp @@ -0,0 +1,68 @@ +/* + * 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 "helpers.cuh" + +#include +#include +#include +#include +#include + +#include +#include + +#include + +namespace cudf::groupby::detail::hash { +/** + * @brief Computes groupby using hash table. + * + * First, we create a hash table that stores the indices of unique rows in + * `keys`. The upper limit on the number of values in this map is the number + * of rows in `keys`. + * + * To store the results of aggregations, we create temporary sparse columns + * which have the same size as input value columns. Using the hash map, we + * determine the location within the sparse column to write the result of the + * aggregation into. + * + * The sparse column results of all aggregations are stored into the cache + * `sparse_results`. This enables the use of previously calculated results in + * other aggregations. + * + * All the aggregations which can be computed in a single pass are computed + * first, in a combined kernel. Then using these results, aggregations that + * require multiple passes, will be computed. + * + * Finally, using the hash map, we generate a vector of indices of populated + * values in sparse result columns. Then, for each aggregation originally + * requested in `requests`, we gather sparse results into a column of dense + * results using the aforementioned index vector. Dense results are stored into + * the in/out parameter `cache`. + */ +template +std::unique_ptr compute_groupby(table_view const& keys, + host_span requests, + bool skip_rows_with_nulls, + Equal const& d_row_equal, + row_hash_t const& d_row_hash, + cudf::detail::result_cache* cache, + rmm::cuda_stream_view stream, + rmm::device_async_resource_ref mr); + +} // namespace cudf::groupby::detail::hash diff --git a/cpp/src/groupby/hash/compute_single_pass_aggs.cu b/cpp/src/groupby/hash/compute_single_pass_aggs.cu new file mode 100644 index 00000000000..58ec48f488e --- /dev/null +++ b/cpp/src/groupby/hash/compute_single_pass_aggs.cu @@ -0,0 +1,131 @@ +/* + * 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 "compute_single_pass_aggs.hpp" +#include "flatten_single_pass_aggs.hpp" +#include "helpers.cuh" +#include "single_pass_functors.cuh" +#include "var_hash_functor.cuh" + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include + +#include +#include +#include + +namespace cudf::groupby::detail::hash { +// make table that will hold sparse results +auto create_sparse_results_table(table_view const& flattened_values, + std::vector aggs, + rmm::cuda_stream_view stream) +{ + // TODO single allocation - room for performance improvement + std::vector> sparse_columns; + std::transform( + flattened_values.begin(), + flattened_values.end(), + aggs.begin(), + std::back_inserter(sparse_columns), + [stream](auto const& col, auto const& agg) { + bool nullable = + (agg == aggregation::COUNT_VALID or agg == aggregation::COUNT_ALL) + ? false + : (col.has_nulls() or agg == aggregation::VARIANCE or agg == aggregation::STD); + auto mask_flag = (nullable) ? mask_state::ALL_NULL : mask_state::UNALLOCATED; + + auto col_type = cudf::is_dictionary(col.type()) + ? cudf::dictionary_column_view(col).keys().type() + : col.type(); + + return make_fixed_width_column( + cudf::detail::target_type(col_type, agg), col.size(), mask_flag, stream); + }); + + table sparse_table(std::move(sparse_columns)); + mutable_table_view table_view = sparse_table.mutable_view(); + cudf::detail::initialize_with_identity(table_view, aggs, stream); + return sparse_table; +} + +/** + * @brief Computes all aggregations from `requests` that require a single pass + * over the data and stores the results in `sparse_results` + */ +template +void compute_single_pass_aggs(int64_t num_keys, + bool skip_rows_with_nulls, + bitmask_type const* row_bitmask, + SetType set, + host_span requests, + cudf::detail::result_cache* sparse_results, + rmm::cuda_stream_view stream) +{ + // flatten the aggs to a table that can be operated on by aggregate_row + auto const [flattened_values, agg_kinds, aggs] = flatten_single_pass_aggs(requests); + + // make table that will hold sparse results + table sparse_table = create_sparse_results_table(flattened_values, agg_kinds, stream); + // prepare to launch kernel to do the actual aggregation + auto d_sparse_table = mutable_table_device_view::create(sparse_table, stream); + auto d_values = table_device_view::create(flattened_values, stream); + auto const d_aggs = cudf::detail::make_device_uvector_async( + agg_kinds, stream, cudf::get_current_device_resource_ref()); + + thrust::for_each_n( + rmm::exec_policy(stream), + thrust::make_counting_iterator(0), + num_keys, + hash::compute_single_pass_aggs_fn{ + set, *d_values, *d_sparse_table, d_aggs.data(), row_bitmask, skip_rows_with_nulls}); + // Add results back to sparse_results cache + auto sparse_result_cols = sparse_table.release(); + for (size_t i = 0; i < aggs.size(); i++) { + // Note that the cache will make a copy of this temporary aggregation + sparse_results->add_result( + flattened_values.column(i), *aggs[i], std::move(sparse_result_cols[i])); + } +} + +template void compute_single_pass_aggs>( + int64_t num_keys, + bool skip_rows_with_nulls, + bitmask_type const* row_bitmask, + hash_set_ref_t set, + host_span requests, + cudf::detail::result_cache* sparse_results, + rmm::cuda_stream_view stream); + +template void compute_single_pass_aggs>( + int64_t num_keys, + bool skip_rows_with_nulls, + bitmask_type const* row_bitmask, + nullable_hash_set_ref_t set, + host_span requests, + cudf::detail::result_cache* sparse_results, + rmm::cuda_stream_view stream); +} // namespace cudf::groupby::detail::hash diff --git a/cpp/src/groupby/hash/compute_single_pass_aggs.hpp b/cpp/src/groupby/hash/compute_single_pass_aggs.hpp new file mode 100644 index 00000000000..a7434bdf61a --- /dev/null +++ b/cpp/src/groupby/hash/compute_single_pass_aggs.hpp @@ -0,0 +1,38 @@ +/* + * 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 +#include +#include +#include + +#include + +namespace cudf::groupby::detail::hash { +/** + * @brief Computes all aggregations from `requests` that require a single pass + * over the data and stores the results in `sparse_results` + */ +template +void compute_single_pass_aggs(int64_t num_keys, + bool skip_rows_with_nulls, + bitmask_type const* row_bitmask, + SetType set, + cudf::host_span requests, + cudf::detail::result_cache* sparse_results, + rmm::cuda_stream_view stream); +} // namespace cudf::groupby::detail::hash diff --git a/cpp/src/groupby/hash/groupby.cu b/cpp/src/groupby/hash/groupby.cu index 0432b9d120a..30e1d52fdbf 100644 --- a/cpp/src/groupby/hash/groupby.cu +++ b/cpp/src/groupby/hash/groupby.cu @@ -14,60 +14,32 @@ * limitations under the License. */ -#include "flatten_single_pass_aggs.hpp" +#include "compute_groupby.hpp" #include "groupby/common/utils.hpp" -#include "groupby_kernels.cuh" -#include "var_hash_functor.cuh" +#include "helpers.cuh" #include -#include -#include -#include -#include -#include #include -#include -#include -#include #include -#include -#include -#include +#include #include #include -#include #include #include -#include #include #include -#include #include #include #include -#include -#include -#include - +#include #include -#include #include +#include -namespace cudf { -namespace groupby { -namespace detail { -namespace hash { +namespace cudf::groupby::detail::hash { namespace { - -// TODO: similar to `contains_table`, using larger CG size like 2 or 4 for nested -// types and `cg_size = 1`for flat data to improve performance -using probing_scheme_type = cuco::linear_probing< - 1, ///< Number of threads used to handle each input key - cudf::experimental::row::hash::device_row_hasher>; - /** * @brief List of aggregation operations that can be computed with a hash-based * implementation. @@ -112,413 +84,33 @@ bool constexpr is_hash_aggregation(aggregation::Kind t) return array_contains(hash_aggregations, t); } -template -class hash_compound_agg_finalizer final : public cudf::detail::aggregation_finalizer { - column_view col; - data_type result_type; - cudf::detail::result_cache* sparse_results; - cudf::detail::result_cache* dense_results; - device_span gather_map; - SetType set; - bitmask_type const* __restrict__ row_bitmask; - rmm::cuda_stream_view stream; - rmm::device_async_resource_ref mr; - - public: - using cudf::detail::aggregation_finalizer::visit; - - hash_compound_agg_finalizer(column_view col, - cudf::detail::result_cache* sparse_results, - cudf::detail::result_cache* dense_results, - device_span gather_map, - SetType set, - bitmask_type const* row_bitmask, - rmm::cuda_stream_view stream, - rmm::device_async_resource_ref mr) - : col(col), - sparse_results(sparse_results), - dense_results(dense_results), - gather_map(gather_map), - set(set), - row_bitmask(row_bitmask), - stream(stream), - mr(mr) - { - result_type = cudf::is_dictionary(col.type()) ? cudf::dictionary_column_view(col).keys().type() - : col.type(); - } - - auto to_dense_agg_result(cudf::aggregation const& agg) - { - auto s = sparse_results->get_result(col, agg); - auto dense_result_table = cudf::detail::gather(table_view({std::move(s)}), - gather_map, - out_of_bounds_policy::DONT_CHECK, - cudf::detail::negative_index_policy::NOT_ALLOWED, - stream, - mr); - return std::move(dense_result_table->release()[0]); - } - - // Enables conversion of ARGMIN/ARGMAX into MIN/MAX - auto gather_argminmax(aggregation const& agg) - { - auto arg_result = to_dense_agg_result(agg); - // We make a view of ARG(MIN/MAX) result without a null mask and gather - // using this map. The values in data buffer of ARG(MIN/MAX) result - // corresponding to null values was initialized to ARG(MIN/MAX)_SENTINEL - // which is an out of bounds index value (-1) and causes the gathered - // value to be null. - column_view null_removed_map( - data_type(type_to_id()), - arg_result->size(), - static_cast(arg_result->view().template data()), - nullptr, - 0); - auto gather_argminmax = - cudf::detail::gather(table_view({col}), - null_removed_map, - arg_result->nullable() ? cudf::out_of_bounds_policy::NULLIFY - : cudf::out_of_bounds_policy::DONT_CHECK, - cudf::detail::negative_index_policy::NOT_ALLOWED, - stream, - mr); - return std::move(gather_argminmax->release()[0]); - } - - // Declare overloads for each kind of aggregation to dispatch - void visit(cudf::aggregation const& agg) override - { - if (dense_results->has_result(col, agg)) return; - dense_results->add_result(col, agg, to_dense_agg_result(agg)); - } - - void visit(cudf::detail::min_aggregation const& agg) override - { - if (dense_results->has_result(col, agg)) return; - if (result_type.id() == type_id::STRING) { - auto transformed_agg = make_argmin_aggregation(); - dense_results->add_result(col, agg, gather_argminmax(*transformed_agg)); - } else { - dense_results->add_result(col, agg, to_dense_agg_result(agg)); - } - } - - void visit(cudf::detail::max_aggregation const& agg) override - { - if (dense_results->has_result(col, agg)) return; - - if (result_type.id() == type_id::STRING) { - auto transformed_agg = make_argmax_aggregation(); - dense_results->add_result(col, agg, gather_argminmax(*transformed_agg)); - } else { - dense_results->add_result(col, agg, to_dense_agg_result(agg)); - } - } - - void visit(cudf::detail::mean_aggregation const& agg) override - { - if (dense_results->has_result(col, agg)) return; - - auto sum_agg = make_sum_aggregation(); - auto count_agg = make_count_aggregation(); - this->visit(*sum_agg); - this->visit(*count_agg); - column_view sum_result = dense_results->get_result(col, *sum_agg); - column_view count_result = dense_results->get_result(col, *count_agg); - - auto result = - cudf::detail::binary_operation(sum_result, - count_result, - binary_operator::DIV, - cudf::detail::target_type(result_type, aggregation::MEAN), - stream, - mr); - dense_results->add_result(col, agg, std::move(result)); - } - - void visit(cudf::detail::var_aggregation const& agg) override - { - if (dense_results->has_result(col, agg)) return; - - auto sum_agg = make_sum_aggregation(); - auto count_agg = make_count_aggregation(); - this->visit(*sum_agg); - this->visit(*count_agg); - column_view sum_result = sparse_results->get_result(col, *sum_agg); - column_view count_result = sparse_results->get_result(col, *count_agg); - - auto values_view = column_device_view::create(col, stream); - auto sum_view = column_device_view::create(sum_result, stream); - auto count_view = column_device_view::create(count_result, stream); - - auto var_result = make_fixed_width_column( - cudf::detail::target_type(result_type, agg.kind), col.size(), mask_state::ALL_NULL, stream); - auto var_result_view = mutable_column_device_view::create(var_result->mutable_view(), stream); - mutable_table_view var_table_view{{var_result->mutable_view()}}; - cudf::detail::initialize_with_identity(var_table_view, {agg.kind}, stream); - - thrust::for_each_n( - rmm::exec_policy(stream), - thrust::make_counting_iterator(0), - col.size(), - var_hash_functor{ - set, row_bitmask, *var_result_view, *values_view, *sum_view, *count_view, agg._ddof}); - sparse_results->add_result(col, agg, std::move(var_result)); - dense_results->add_result(col, agg, to_dense_agg_result(agg)); - } - - void visit(cudf::detail::std_aggregation const& agg) override - { - if (dense_results->has_result(col, agg)) return; - auto var_agg = make_variance_aggregation(agg._ddof); - this->visit(*dynamic_cast(var_agg.get())); - column_view variance = dense_results->get_result(col, *var_agg); - - auto result = cudf::detail::unary_operation(variance, unary_operator::SQRT, stream, mr); - dense_results->add_result(col, agg, std::move(result)); - } -}; - -/** - * @brief Gather sparse results into dense using `gather_map` and add to - * `dense_cache` - * - * @see groupby_null_templated() - */ -template -void sparse_to_dense_results(table_view const& keys, - host_span requests, - cudf::detail::result_cache* sparse_results, - cudf::detail::result_cache* dense_results, - device_span gather_map, - SetType set, - bool keys_have_nulls, - null_policy include_null_keys, - rmm::cuda_stream_view stream, - rmm::device_async_resource_ref mr) -{ - auto row_bitmask = - cudf::detail::bitmask_and(keys, stream, cudf::get_current_device_resource_ref()).first; - bool skip_key_rows_with_nulls = keys_have_nulls and include_null_keys == null_policy::EXCLUDE; - bitmask_type const* row_bitmask_ptr = - skip_key_rows_with_nulls ? static_cast(row_bitmask.data()) : nullptr; - - for (auto const& request : requests) { - auto const& agg_v = request.aggregations; - auto const& col = request.values; - - // Given an aggregation, this will get the result from sparse_results and - // convert and return dense, compacted result - auto finalizer = hash_compound_agg_finalizer( - col, sparse_results, dense_results, gather_map, set, row_bitmask_ptr, stream, mr); - for (auto&& agg : agg_v) { - agg->finalize(finalizer); - } - } -} - -// make table that will hold sparse results -auto create_sparse_results_table(table_view const& flattened_values, - std::vector aggs, - rmm::cuda_stream_view stream) -{ - // TODO single allocation - room for performance improvement - std::vector> sparse_columns; - std::transform( - flattened_values.begin(), - flattened_values.end(), - aggs.begin(), - std::back_inserter(sparse_columns), - [stream](auto const& col, auto const& agg) { - bool nullable = - (agg == aggregation::COUNT_VALID or agg == aggregation::COUNT_ALL) - ? false - : (col.has_nulls() or agg == aggregation::VARIANCE or agg == aggregation::STD); - auto mask_flag = (nullable) ? mask_state::ALL_NULL : mask_state::UNALLOCATED; - - auto col_type = cudf::is_dictionary(col.type()) - ? cudf::dictionary_column_view(col).keys().type() - : col.type(); - - return make_fixed_width_column( - cudf::detail::target_type(col_type, agg), col.size(), mask_flag, stream); - }); - - table sparse_table(std::move(sparse_columns)); - mutable_table_view table_view = sparse_table.mutable_view(); - cudf::detail::initialize_with_identity(table_view, aggs, stream); - return sparse_table; -} - -/** - * @brief Computes all aggregations from `requests` that require a single pass - * over the data and stores the results in `sparse_results` - */ -template -void compute_single_pass_aggs(table_view const& keys, - host_span requests, - cudf::detail::result_cache* sparse_results, - SetType set, - bool keys_have_nulls, - null_policy include_null_keys, - rmm::cuda_stream_view stream) -{ - // flatten the aggs to a table that can be operated on by aggregate_row - auto const [flattened_values, agg_kinds, aggs] = flatten_single_pass_aggs(requests); - - // make table that will hold sparse results - table sparse_table = create_sparse_results_table(flattened_values, agg_kinds, stream); - // prepare to launch kernel to do the actual aggregation - auto d_sparse_table = mutable_table_device_view::create(sparse_table, stream); - auto d_values = table_device_view::create(flattened_values, stream); - auto const d_aggs = cudf::detail::make_device_uvector_async( - agg_kinds, stream, cudf::get_current_device_resource_ref()); - auto const skip_key_rows_with_nulls = - keys_have_nulls and include_null_keys == null_policy::EXCLUDE; - - auto row_bitmask = - skip_key_rows_with_nulls - ? cudf::detail::bitmask_and(keys, stream, cudf::get_current_device_resource_ref()).first - : rmm::device_buffer{}; - - thrust::for_each_n( - rmm::exec_policy(stream), - thrust::make_counting_iterator(0), - keys.num_rows(), - hash::compute_single_pass_aggs_fn{set, - *d_values, - *d_sparse_table, - d_aggs.data(), - static_cast(row_bitmask.data()), - skip_key_rows_with_nulls}); - // Add results back to sparse_results cache - auto sparse_result_cols = sparse_table.release(); - for (size_t i = 0; i < aggs.size(); i++) { - // Note that the cache will make a copy of this temporary aggregation - sparse_results->add_result( - flattened_values.column(i), *aggs[i], std::move(sparse_result_cols[i])); - } -} - -/** - * @brief Computes and returns a device vector containing all populated keys in - * `map`. - */ -template -rmm::device_uvector extract_populated_keys(SetType const& key_set, - size_type num_keys, - rmm::cuda_stream_view stream) -{ - rmm::device_uvector populated_keys(num_keys, stream); - auto const keys_end = key_set.retrieve_all(populated_keys.begin(), stream.value()); - - populated_keys.resize(std::distance(populated_keys.begin(), keys_end), stream); - return populated_keys; -} - -/** - * @brief Computes groupby using hash table. - * - * First, we create a hash table that stores the indices of unique rows in - * `keys`. The upper limit on the number of values in this map is the number - * of rows in `keys`. - * - * To store the results of aggregations, we create temporary sparse columns - * which have the same size as input value columns. Using the hash map, we - * determine the location within the sparse column to write the result of the - * aggregation into. - * - * The sparse column results of all aggregations are stored into the cache - * `sparse_results`. This enables the use of previously calculated results in - * other aggregations. - * - * All the aggregations which can be computed in a single pass are computed - * first, in a combined kernel. Then using these results, aggregations that - * require multiple passes, will be computed. - * - * Finally, using the hash map, we generate a vector of indices of populated - * values in sparse result columns. Then, for each aggregation originally - * requested in `requests`, we gather sparse results into a column of dense - * results using the aforementioned index vector. Dense results are stored into - * the in/out parameter `cache`. - */ -std::unique_ptr
groupby(table_view const& keys, - host_span requests, - cudf::detail::result_cache* cache, - bool const keys_have_nulls, - null_policy const include_null_keys, - rmm::cuda_stream_view stream, - rmm::device_async_resource_ref mr) +std::unique_ptr
dispatch_groupby(table_view const& keys, + host_span requests, + cudf::detail::result_cache* cache, + bool const keys_have_nulls, + null_policy const include_null_keys, + rmm::cuda_stream_view stream, + rmm::device_async_resource_ref mr) { - // convert to int64_t to avoid potential overflow with large `keys` - auto const num_keys = static_cast(keys.num_rows()); - auto const null_keys_are_equal = null_equality::EQUAL; - auto const has_null = nullate::DYNAMIC{cudf::has_nested_nulls(keys)}; + auto const null_keys_are_equal = null_equality::EQUAL; + auto const has_null = nullate::DYNAMIC{cudf::has_nested_nulls(keys)}; + auto const skip_rows_with_nulls = keys_have_nulls and include_null_keys == null_policy::EXCLUDE; auto preprocessed_keys = cudf::experimental::row::hash::preprocessed_table::create(keys, stream); auto const comparator = cudf::experimental::row::equality::self_comparator{preprocessed_keys}; auto const row_hash = cudf::experimental::row::hash::row_hasher{std::move(preprocessed_keys)}; auto const d_row_hash = row_hash.device_hasher(has_null); - // Cache of sparse results where the location of aggregate value in each - // column is indexed by the hash set - cudf::detail::result_cache sparse_results(requests.size()); - - auto const comparator_helper = [&](auto const d_key_equal) { - auto const set = cuco::static_set{ - num_keys, - 0.5, // desired load factor - cuco::empty_key{cudf::detail::CUDF_SIZE_TYPE_SENTINEL}, - d_key_equal, - probing_scheme_type{d_row_hash}, - cuco::thread_scope_device, - cuco::storage<1>{}, - cudf::detail::cuco_allocator{rmm::mr::polymorphic_allocator{}, stream}, - stream.value()}; - - // Compute all single pass aggs first - compute_single_pass_aggs(keys, - requests, - &sparse_results, - set.ref(cuco::insert_and_find), - keys_have_nulls, - include_null_keys, - stream); - - // Extract the populated indices from the hash set and create a gather map. - // Gathering using this map from sparse results will give dense results. - auto gather_map = extract_populated_keys(set, keys.num_rows(), stream); - - // Compact all results from sparse_results and insert into cache - sparse_to_dense_results(keys, - requests, - &sparse_results, - cache, - gather_map, - set.ref(cuco::find), - keys_have_nulls, - include_null_keys, - stream, - mr); - - return cudf::detail::gather(keys, - gather_map, - out_of_bounds_policy::DONT_CHECK, - cudf::detail::negative_index_policy::NOT_ALLOWED, - stream, - mr); - }; - if (cudf::detail::has_nested_columns(keys)) { - auto const d_key_equal = comparator.equal_to(has_null, null_keys_are_equal); - return comparator_helper(d_key_equal); + auto const d_row_equal = comparator.equal_to(has_null, null_keys_are_equal); + return compute_groupby( + keys, requests, skip_rows_with_nulls, d_row_equal, d_row_hash, cache, stream, mr); } else { - auto const d_key_equal = comparator.equal_to(has_null, null_keys_are_equal); - return comparator_helper(d_key_equal); + auto const d_row_equal = comparator.equal_to(has_null, null_keys_are_equal); + return compute_groupby( + keys, requests, skip_rows_with_nulls, d_row_equal, d_row_hash, cache, stream, mr); } } - } // namespace /** @@ -559,11 +151,8 @@ std::pair, std::vector> groupby( cudf::detail::result_cache cache(requests.size()); std::unique_ptr
unique_keys = - groupby(keys, requests, &cache, cudf::has_nulls(keys), include_null_keys, stream, mr); + dispatch_groupby(keys, requests, &cache, cudf::has_nulls(keys), include_null_keys, stream, mr); return std::pair(std::move(unique_keys), extract_results(requests, cache, stream, mr)); } -} // namespace hash -} // namespace detail -} // namespace groupby -} // namespace cudf +} // namespace cudf::groupby::detail::hash diff --git a/cpp/src/groupby/hash/hash_compound_agg_finalizer.cu b/cpp/src/groupby/hash/hash_compound_agg_finalizer.cu new file mode 100644 index 00000000000..41d00ee6d37 --- /dev/null +++ b/cpp/src/groupby/hash/hash_compound_agg_finalizer.cu @@ -0,0 +1,199 @@ +/* + * 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 "hash_compound_agg_finalizer.hpp" +#include "helpers.cuh" +#include "var_hash_functor.cuh" + +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include + +#include + +namespace cudf::groupby::detail::hash { +template +hash_compound_agg_finalizer::hash_compound_agg_finalizer( + column_view col, + cudf::detail::result_cache* sparse_results, + cudf::detail::result_cache* dense_results, + device_span gather_map, + SetType set, + bitmask_type const* row_bitmask, + rmm::cuda_stream_view stream, + rmm::device_async_resource_ref mr) + : col(col), + sparse_results(sparse_results), + dense_results(dense_results), + gather_map(gather_map), + set(set), + row_bitmask(row_bitmask), + stream(stream), + mr(mr) +{ + result_type = + cudf::is_dictionary(col.type()) ? cudf::dictionary_column_view(col).keys().type() : col.type(); +} + +template +auto hash_compound_agg_finalizer::to_dense_agg_result(cudf::aggregation const& agg) +{ + auto s = sparse_results->get_result(col, agg); + auto dense_result_table = cudf::detail::gather(table_view({std::move(s)}), + gather_map, + out_of_bounds_policy::DONT_CHECK, + cudf::detail::negative_index_policy::NOT_ALLOWED, + stream, + mr); + return std::move(dense_result_table->release()[0]); +} + +template +auto hash_compound_agg_finalizer::gather_argminmax(aggregation const& agg) +{ + auto arg_result = to_dense_agg_result(agg); + // We make a view of ARG(MIN/MAX) result without a null mask and gather + // using this map. The values in data buffer of ARG(MIN/MAX) result + // corresponding to null values was initialized to ARG(MIN/MAX)_SENTINEL + // which is an out of bounds index value (-1) and causes the gathered + // value to be null. + column_view null_removed_map( + data_type(type_to_id()), + arg_result->size(), + static_cast(arg_result->view().template data()), + nullptr, + 0); + auto gather_argminmax = + cudf::detail::gather(table_view({col}), + null_removed_map, + arg_result->nullable() ? cudf::out_of_bounds_policy::NULLIFY + : cudf::out_of_bounds_policy::DONT_CHECK, + cudf::detail::negative_index_policy::NOT_ALLOWED, + stream, + mr); + return std::move(gather_argminmax->release()[0]); +} + +template +void hash_compound_agg_finalizer::visit(cudf::aggregation const& agg) +{ + if (dense_results->has_result(col, agg)) return; + dense_results->add_result(col, agg, to_dense_agg_result(agg)); +} + +template +void hash_compound_agg_finalizer::visit(cudf::detail::min_aggregation const& agg) +{ + if (dense_results->has_result(col, agg)) return; + if (result_type.id() == type_id::STRING) { + auto transformed_agg = make_argmin_aggregation(); + dense_results->add_result(col, agg, gather_argminmax(*transformed_agg)); + } else { + dense_results->add_result(col, agg, to_dense_agg_result(agg)); + } +} + +template +void hash_compound_agg_finalizer::visit(cudf::detail::max_aggregation const& agg) +{ + if (dense_results->has_result(col, agg)) return; + + if (result_type.id() == type_id::STRING) { + auto transformed_agg = make_argmax_aggregation(); + dense_results->add_result(col, agg, gather_argminmax(*transformed_agg)); + } else { + dense_results->add_result(col, agg, to_dense_agg_result(agg)); + } +} + +template +void hash_compound_agg_finalizer::visit(cudf::detail::mean_aggregation const& agg) +{ + if (dense_results->has_result(col, agg)) return; + + auto sum_agg = make_sum_aggregation(); + auto count_agg = make_count_aggregation(); + this->visit(*sum_agg); + this->visit(*count_agg); + column_view sum_result = dense_results->get_result(col, *sum_agg); + column_view count_result = dense_results->get_result(col, *count_agg); + + auto result = + cudf::detail::binary_operation(sum_result, + count_result, + binary_operator::DIV, + cudf::detail::target_type(result_type, aggregation::MEAN), + stream, + mr); + dense_results->add_result(col, agg, std::move(result)); +} + +template +void hash_compound_agg_finalizer::visit(cudf::detail::var_aggregation const& agg) +{ + if (dense_results->has_result(col, agg)) return; + + auto sum_agg = make_sum_aggregation(); + auto count_agg = make_count_aggregation(); + this->visit(*sum_agg); + this->visit(*count_agg); + column_view sum_result = sparse_results->get_result(col, *sum_agg); + column_view count_result = sparse_results->get_result(col, *count_agg); + + auto values_view = column_device_view::create(col, stream); + auto sum_view = column_device_view::create(sum_result, stream); + auto count_view = column_device_view::create(count_result, stream); + + auto var_result = make_fixed_width_column( + cudf::detail::target_type(result_type, agg.kind), col.size(), mask_state::ALL_NULL, stream); + auto var_result_view = mutable_column_device_view::create(var_result->mutable_view(), stream); + mutable_table_view var_table_view{{var_result->mutable_view()}}; + cudf::detail::initialize_with_identity(var_table_view, {agg.kind}, stream); + + thrust::for_each_n( + rmm::exec_policy(stream), + thrust::make_counting_iterator(0), + col.size(), + var_hash_functor{ + set, row_bitmask, *var_result_view, *values_view, *sum_view, *count_view, agg._ddof}); + sparse_results->add_result(col, agg, std::move(var_result)); + dense_results->add_result(col, agg, to_dense_agg_result(agg)); +} + +template +void hash_compound_agg_finalizer::visit(cudf::detail::std_aggregation const& agg) +{ + if (dense_results->has_result(col, agg)) return; + auto var_agg = make_variance_aggregation(agg._ddof); + this->visit(*dynamic_cast(var_agg.get())); + column_view variance = dense_results->get_result(col, *var_agg); + + auto result = cudf::detail::unary_operation(variance, unary_operator::SQRT, stream, mr); + dense_results->add_result(col, agg, std::move(result)); +} + +template class hash_compound_agg_finalizer>; +template class hash_compound_agg_finalizer>; +} // namespace cudf::groupby::detail::hash diff --git a/cpp/src/groupby/hash/hash_compound_agg_finalizer.hpp b/cpp/src/groupby/hash/hash_compound_agg_finalizer.hpp new file mode 100644 index 00000000000..8bee1a92c40 --- /dev/null +++ b/cpp/src/groupby/hash/hash_compound_agg_finalizer.hpp @@ -0,0 +1,69 @@ +/* + * 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 +#include +#include +#include + +#include +#include + +namespace cudf::groupby::detail::hash { +template +class hash_compound_agg_finalizer final : public cudf::detail::aggregation_finalizer { + column_view col; + data_type result_type; + cudf::detail::result_cache* sparse_results; + cudf::detail::result_cache* dense_results; + device_span gather_map; + SetType set; + bitmask_type const* __restrict__ row_bitmask; + rmm::cuda_stream_view stream; + rmm::device_async_resource_ref mr; + + public: + using cudf::detail::aggregation_finalizer::visit; + + hash_compound_agg_finalizer(column_view col, + cudf::detail::result_cache* sparse_results, + cudf::detail::result_cache* dense_results, + device_span gather_map, + SetType set, + bitmask_type const* row_bitmask, + rmm::cuda_stream_view stream, + rmm::device_async_resource_ref mr); + + auto to_dense_agg_result(cudf::aggregation const& agg); + + // Enables conversion of ARGMIN/ARGMAX into MIN/MAX + auto gather_argminmax(cudf::aggregation const& agg); + + // Declare overloads for each kind of aggregation to dispatch + void visit(cudf::aggregation const& agg) override; + + void visit(cudf::detail::min_aggregation const& agg) override; + + void visit(cudf::detail::max_aggregation const& agg) override; + + void visit(cudf::detail::mean_aggregation const& agg) override; + + void visit(cudf::detail::var_aggregation const& agg) override; + + void visit(cudf::detail::std_aggregation const& agg) override; +}; +} // namespace cudf::groupby::detail::hash diff --git a/cpp/src/groupby/hash/helpers.cuh b/cpp/src/groupby/hash/helpers.cuh new file mode 100644 index 00000000000..0d117ca35b3 --- /dev/null +++ b/cpp/src/groupby/hash/helpers.cuh @@ -0,0 +1,116 @@ +/* + * 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 +#include +#include +#include + +#include + +namespace cudf::groupby::detail::hash { +// TODO: similar to `contains_table`, using larger CG size like 2 or 4 for nested +// types and `cg_size = 1`for flat data to improve performance +/// Number of threads to handle each input element +CUDF_HOST_DEVICE auto constexpr GROUPBY_CG_SIZE = 1; + +/// Number of slots per thread +CUDF_HOST_DEVICE auto constexpr GROUPBY_WINDOW_SIZE = 1; + +/// Thread block size +CUDF_HOST_DEVICE auto constexpr GROUPBY_BLOCK_SIZE = 128; + +/// Threshold cardinality to switch between shared memory aggregations and global memory +/// aggregations +CUDF_HOST_DEVICE auto constexpr GROUPBY_CARDINALITY_THRESHOLD = 128; + +// We add additional `block_size`, because after the number of elements in the local hash set +// exceeds the threshold, all threads in the thread block can still insert one more element. +/// The maximum number of elements handled per block +CUDF_HOST_DEVICE auto constexpr GROUPBY_SHM_MAX_ELEMENTS = + GROUPBY_CARDINALITY_THRESHOLD + GROUPBY_BLOCK_SIZE; + +// GROUPBY_SHM_MAX_ELEMENTS with 0.7 occupancy +/// Shared memory hash set extent type +using shmem_extent_t = + cuco::extent(static_cast(GROUPBY_SHM_MAX_ELEMENTS) * 1.43)>; + +/// Number of windows needed by each shared memory hash set +CUDF_HOST_DEVICE auto constexpr window_extent = + cuco::make_window_extent(shmem_extent_t{}); + +/** + * @brief Returns the smallest multiple of 8 that is greater than or equal to the given integer. + */ +CUDF_HOST_DEVICE constexpr std::size_t round_to_multiple_of_8(std::size_t num) +{ + std::size_t constexpr base = 8; + return cudf::util::div_rounding_up_safe(num, base) * base; +} + +using row_hash_t = + cudf::experimental::row::hash::device_row_hasher; + +/// Probing scheme type used by groupby hash table +using probing_scheme_t = cuco::linear_probing; + +using row_comparator_t = cudf::experimental::row::equality::device_row_comparator< + false, + cudf::nullate::DYNAMIC, + cudf::experimental::row::equality::nan_equal_physical_equality_comparator>; + +using nullable_row_comparator_t = cudf::experimental::row::equality::device_row_comparator< + true, + cudf::nullate::DYNAMIC, + cudf::experimental::row::equality::nan_equal_physical_equality_comparator>; + +using global_set_t = cuco::static_set, + cuda::thread_scope_device, + row_comparator_t, + probing_scheme_t, + cudf::detail::cuco_allocator, + cuco::storage>; + +using nullable_global_set_t = cuco::static_set, + cuda::thread_scope_device, + nullable_row_comparator_t, + probing_scheme_t, + cudf::detail::cuco_allocator, + cuco::storage>; + +template +using hash_set_ref_t = cuco::static_set_ref< + cudf::size_type, + cuda::thread_scope_device, + row_comparator_t, + probing_scheme_t, + cuco::aow_storage_ref>, + Op>; + +template +using nullable_hash_set_ref_t = cuco::static_set_ref< + cudf::size_type, + cuda::thread_scope_device, + nullable_row_comparator_t, + probing_scheme_t, + cuco::aow_storage_ref>, + Op>; +} // namespace cudf::groupby::detail::hash diff --git a/cpp/src/groupby/hash/groupby_kernels.cuh b/cpp/src/groupby/hash/single_pass_functors.cuh similarity index 95% rename from cpp/src/groupby/hash/groupby_kernels.cuh rename to cpp/src/groupby/hash/single_pass_functors.cuh index 86f4d76487f..73791b3aa71 100644 --- a/cpp/src/groupby/hash/groupby_kernels.cuh +++ b/cpp/src/groupby/hash/single_pass_functors.cuh @@ -13,7 +13,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - #pragma once #include @@ -21,12 +20,9 @@ #include #include -#include +#include -namespace cudf { -namespace groupby { -namespace detail { -namespace hash { +namespace cudf::groupby::detail::hash { /** * @brief Computes single-pass aggregations and store results into a sparse `output_values` table, * and populate `set` with indices of unique keys @@ -102,8 +98,4 @@ struct compute_single_pass_aggs_fn { } } }; - -} // namespace hash -} // namespace detail -} // namespace groupby -} // namespace cudf +} // namespace cudf::groupby::detail::hash diff --git a/cpp/src/groupby/hash/sparse_to_dense_results.cu b/cpp/src/groupby/hash/sparse_to_dense_results.cu new file mode 100644 index 00000000000..82a7945de71 --- /dev/null +++ b/cpp/src/groupby/hash/sparse_to_dense_results.cu @@ -0,0 +1,72 @@ +/* + * 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 "hash_compound_agg_finalizer.hpp" +#include "helpers.cuh" + +#include +#include +#include +#include + +#include +#include + +namespace cudf::groupby::detail::hash { +template +void sparse_to_dense_results(host_span requests, + cudf::detail::result_cache* sparse_results, + cudf::detail::result_cache* dense_results, + device_span gather_map, + SetType set, + bitmask_type const* row_bitmask, + rmm::cuda_stream_view stream, + rmm::device_async_resource_ref mr) +{ + for (auto const& request : requests) { + auto const& agg_v = request.aggregations; + auto const& col = request.values; + + // Given an aggregation, this will get the result from sparse_results and + // convert and return dense, compacted result + auto finalizer = hash_compound_agg_finalizer( + col, sparse_results, dense_results, gather_map, set, row_bitmask, stream, mr); + for (auto&& agg : agg_v) { + agg->finalize(finalizer); + } + } +} + +template void sparse_to_dense_results>( + host_span requests, + cudf::detail::result_cache* sparse_results, + cudf::detail::result_cache* dense_results, + device_span gather_map, + hash_set_ref_t set, + bitmask_type const* row_bitmask, + rmm::cuda_stream_view stream, + rmm::device_async_resource_ref mr); + +template void sparse_to_dense_results>( + host_span requests, + cudf::detail::result_cache* sparse_results, + cudf::detail::result_cache* dense_results, + device_span gather_map, + nullable_hash_set_ref_t set, + bitmask_type const* row_bitmask, + rmm::cuda_stream_view stream, + rmm::device_async_resource_ref mr); +} // namespace cudf::groupby::detail::hash diff --git a/cpp/src/groupby/hash/sparse_to_dense_results.hpp b/cpp/src/groupby/hash/sparse_to_dense_results.hpp new file mode 100644 index 00000000000..ca252546ecd --- /dev/null +++ b/cpp/src/groupby/hash/sparse_to_dense_results.hpp @@ -0,0 +1,42 @@ +/* + * 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 +#include +#include +#include + +#include +#include + +namespace cudf::groupby::detail::hash { +/** + * @brief Gather sparse results into dense using `gather_map` and add to + * `dense_cache` + * + * @see groupby_null_templated() + */ +template +void sparse_to_dense_results(host_span requests, + cudf::detail::result_cache* sparse_results, + cudf::detail::result_cache* dense_results, + device_span gather_map, + SetType set, + bitmask_type const* row_bitmask, + rmm::cuda_stream_view stream, + rmm::device_async_resource_ref mr); +} // namespace cudf::groupby::detail::hash