diff --git a/src/main/cpp/CMakeLists.txt b/src/main/cpp/CMakeLists.txt index 838e4c34e..3d3875b9b 100644 --- a/src/main/cpp/CMakeLists.txt +++ b/src/main/cpp/CMakeLists.txt @@ -199,6 +199,7 @@ add_library( src/HashJni.cpp src/HistogramJni.cpp src/HostTableJni.cpp + src/HyperLogLogPlusPlusHostUDFJni.cpp src/JSONUtilsJni.cpp src/NativeParquetJni.cpp src/ParseURIJni.cpp @@ -221,6 +222,9 @@ add_library( src/from_json_to_structs.cu src/get_json_object.cu src/histogram.cu + src/hive_hash.cu + src/hyper_log_log_plus_plus.cu + src/hyper_log_log_plus_plus_host_udf.cu src/json_utils.cu src/murmur_hash.cu src/parse_uri.cu @@ -230,7 +234,6 @@ add_library( src/timezones.cu src/utilities.cu src/xxhash64.cu - src/hive_hash.cu src/zorder.cu ) diff --git a/src/main/cpp/src/HyperLogLogPlusPlusHostUDFJni.cpp b/src/main/cpp/src/HyperLogLogPlusPlusHostUDFJni.cpp new file mode 100644 index 000000000..1e1efc05f --- /dev/null +++ b/src/main/cpp/src/HyperLogLogPlusPlusHostUDFJni.cpp @@ -0,0 +1,63 @@ +/* + * Copyright (c) 2024-2025, 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 "cudf_jni_apis.hpp" +#include "hyper_log_log_plus_plus.hpp" +#include "hyper_log_log_plus_plus_host_udf.hpp" + +extern "C" { + +JNIEXPORT jlong JNICALL +Java_com_nvidia_spark_rapids_jni_HyperLogLogPlusPlusHostUDF_createHLLPPHostUDF(JNIEnv* env, + jclass, + jint agg_type, + int precision) +{ + try { + cudf::jni::auto_set_device(env); + auto udf_ptr = [&] { + // The value of agg_type must be sync with + // `HyperLogLogPlusPlusHostUDF.java#AggregationType`. + switch (agg_type) { + case 0: return spark_rapids_jni::create_hllpp_reduction_host_udf(precision); + case 1: return spark_rapids_jni::create_hllpp_reduction_merge_host_udf(precision); + case 2: return spark_rapids_jni::create_hllpp_groupby_host_udf(precision); + case 3: return spark_rapids_jni::create_hllpp_groupby_merge_host_udf(precision); + default: CUDF_FAIL("Invalid aggregation type."); + } + }(); + CUDF_EXPECTS(udf_ptr != nullptr, "Invalid HyperLogLogPlusPlus(HLLPP) UDF instance."); + + return reinterpret_cast(udf_ptr); + } + CATCH_STD(env, 0); +} + +JNIEXPORT jlong JNICALL +Java_com_nvidia_spark_rapids_jni_HyperLogLogPlusPlusHostUDF_estimateDistinctValueFromSketches( + JNIEnv* env, jclass, jlong sketches, jint precision) +{ + JNI_NULL_CHECK(env, sketches, "Sketch column is null", 0); + try { + cudf::jni::auto_set_device(env); + auto const sketch_view = reinterpret_cast(sketches); + return cudf::jni::ptr_as_jlong( + spark_rapids_jni::estimate_from_hll_sketches(*sketch_view, precision).release()); + } + CATCH_STD(env, 0); +} + +} // extern "C" diff --git a/src/main/cpp/src/hyper_log_log_plus_plus.cu b/src/main/cpp/src/hyper_log_log_plus_plus.cu new file mode 100644 index 000000000..9f9828656 --- /dev/null +++ b/src/main/cpp/src/hyper_log_log_plus_plus.cu @@ -0,0 +1,964 @@ +/* + * Copyright (c) 2024-2025, 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.hpp" +#include "hyper_log_log_plus_plus.hpp" +#include "hyper_log_log_plus_plus_const.hpp" + +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include + +#include +#include +#include // TODO #include once available +#include +#include +#include + +namespace spark_rapids_jni { + +namespace { + +/** + * @brief The seed used for the XXHash64 hash function. + * It's consistent with Spark + */ +constexpr int64_t SEED = 42L; + +/** + * @brief 6 binary MASK bits: 111-111 + */ +constexpr uint64_t MASK = (1L << REGISTER_VALUE_BITS) - 1L; + +/** + * @brief The maximum precision that can be used for the HLLPP algorithm. + * If input precision is bigger than 18, then use 18. + */ +constexpr int MAX_PRECISION = 18; + +/** + * @brief Get register value from a long which contains 10 register values, + * each register value in long is 6 bits. + */ +__device__ inline int get_register_value(int64_t const ten_registers, int reg_idx) +{ + auto const shift_bits = REGISTER_VALUE_BITS * reg_idx; + auto const shift_mask = MASK << shift_bits; + auto const v = (ten_registers & shift_mask) >> shift_bits; + return static_cast(v); +} + +/** + * @brief Computes HLLPP sketches(register values) from hash values and + * partially merge the sketches. + * + * Tried to use `reduce_by_key`, but it uses too much of memory, so give up using `reduce_by_key`. + * More details: + * `reduce_by_key` uses num_rows_input intermidate cache: + * https://github.com/NVIDIA/thrust/blob/2.1.0/thrust/system/detail/generic/reduce_by_key.inl#L112 + * // scan the values by flag + * thrust::detail::temporary_array + * scanned_values(exec, n); + * Each sketch contains multiple integers, by default 512 integers(precision is + * 9), num_rows_input * 512 is huge. + * + * This function uses a differrent approach to use less intermidate cache. + * It uses 2 phase merges: partial merge and final merge + * + * This function splits input into multiple segments with each segment has + * num_hashs_per_thread items. The input is sorted by group labels, each segment + * contains one or more consecutive groups. Each thread handles one segment with + * num_hashs_per_thread items in it: + * - Scan all the items in the segment, update the max value. + * - Output max value into registers_output_cache for the previous group when + * meets a new group. + * - Output max value into registers_thread_cache when reach the last item in + * the segment. + * + * In this way, we can save memory usage, cache less intermidate sketches + * (num_hashs / num_hashs_per_thread) sketches. + * num_threads = div_round_up(num_hashs, num_hashs_per_thread). + * + * Note: Must exclude null hash values from computing HLLPP sketches. + * + * e.g.: num_registers_per_sketch = 512 and num_hashs_per_thread = 4; + * + * Input is hashs, compute and get pair: register index -> register value + * + * reg_index, reg_value, group_label + * [ + * ---------- segment 0 begin -------------------------- + * (0, 1), g0 + * (0, 2), g0 + * // meets new group g1, save result for group g0 into registers_output_cache + * (1, 1), g1 + * // outputs result at segemnt end for this thread to registers_thread_cache + * (1, 9), g1 + * ---------- segment 1 begin -------------------------- + * (1, 1), g1 + * (1, 1), g1 + * (1, 5), g1 + * // outputs result at segemnt end for this thread to registers_thread_cache + * (1, 1), g1 + * ---------- segment 2 begin -------------------------- + * (1, 1), g1 + * (1, 1), g1 + * (1, 8), g1 + * // outputs result at segemnt end for this thread to registers_thread_cache + * // assumes meets new group when at the end, save to registers_output_cache + * (1, 1), g1 + * ] + * Output e.g.: + * + * group_labels_thread_cache: + * [ + * g1 + * g1 + * g1 + * ] + * Has num_threads rows. + * + * registers_thread_cache: + * [ + * 512 values: [0, 9, 0, ... ] // register values for group 1 + * 512 values: [0, 5, 0, ... ] // register values for group 1 + * 512 values: [0, 8, 0, ... ] // register values for group 1 + * ] + * Has num_threads rows, each row is corresponding to + * `group_labels_thread_cache` + * + * registers_output_cache: + * [ + * 512 values: [2, 0, 0, ... ] // register values for group 0 + * 512 values: [0, 8, 0, ... ] // register values for group 1 + * ] + * Has num_groups rows. + * + * The next kernel will merge the registers_output_cache and + * registers_thread_cache and get the final result. + */ +template +CUDF_KERNEL void partial_group_sketches_from_hashs_kernel( + cudf::column_device_view hashs, + cudf::device_span group_labels, + int64_t const precision, // num of bits for register addressing, e.g.: 9 + int* const registers_output_cache, // num is num_groups * num_registers_per_sketch + int* const registers_thread_cache, // num is num_threads * num_registers_per_sketch + cudf::size_type* const group_labels_thread_cache // save the group labels for each thread +) +{ + auto const tid = cudf::detail::grid_1d::global_thread_id(); + int64_t const num_hashs = hashs.size(); + if (tid * num_hashs_per_thread >= hashs.size()) { return; } + + // 2^precision = num_registers_per_sketch + int64_t num_registers_per_sketch = 1L << precision; + // e.g.: integer in binary: 1 0000 0000 + uint64_t const w_padding = 1ULL << (precision - 1); + // e.g.: 64 - 9 = 55 + int const idx_shift = 64 - precision; + + auto const hash_first = tid * num_hashs_per_thread; + auto const hash_end = cuda::std::min((tid + 1) * num_hashs_per_thread, num_hashs); + + // init sketches for each thread + int* const sketch_ptr = registers_thread_cache + tid * num_registers_per_sketch; + for (auto i = 0; i < num_registers_per_sketch; i++) { + sketch_ptr[i] = 0; + } + + cudf::size_type prev_group = group_labels[hash_first]; + for (auto hash_idx = hash_first; hash_idx < hash_end; hash_idx++) { + cudf::size_type curr_group = group_labels[hash_idx]; + + int reg_idx = 0; // init value for null hash + int reg_v = 0; // init value for null hash + if (!hashs.is_null(hash_idx)) { + // cast to unsigned, then >> will shift without preserve the sign bit. + uint64_t const hash = static_cast(hashs.element(hash_idx)); + reg_idx = hash >> idx_shift; + // get the leading zeros + reg_v = static_cast(cuda::std::countl_zero((hash << precision) | w_padding) + 1ULL); + } + + if (curr_group == prev_group) { + // still in the same group, update the max value + if (reg_v > sketch_ptr[reg_idx]) { sketch_ptr[reg_idx] = reg_v; } + } else { + // meets new group, save output for the previous group and reset + for (auto i = 0; i < num_registers_per_sketch; i++) { + registers_output_cache[prev_group * num_registers_per_sketch + i] = sketch_ptr[i]; + sketch_ptr[i] = 0; + } + // save the result for current group + sketch_ptr[reg_idx] = reg_v; + } + + if (hash_idx == hash_end - 1) { + // meets the last hash in the segment + if (hash_idx == num_hashs - 1) { + // meets the last segment, special logic: assume meets new group + for (auto i = 0; i < num_registers_per_sketch; i++) { + registers_output_cache[curr_group * num_registers_per_sketch + i] = sketch_ptr[i]; + } + } else { + // not the last segment, probe one item forward. + if (curr_group != group_labels[hash_idx + 1]) { + // meets a new group by checking the next item in the next segment + for (auto i = 0; i < num_registers_per_sketch; i++) { + registers_output_cache[curr_group * num_registers_per_sketch + i] = sketch_ptr[i]; + } + } + } + } + + prev_group = curr_group; + } + + // save the group label for this thread + group_labels_thread_cache[tid] = group_labels[hash_end - 1]; +} + +/* + * @brief Merge registers_thread_cache into registers_output_cache, both of them + * are produced in the above kernel. Merge sketches vertically. + * + * For each register index, starts a thread to merge registers in + * registers_thread_cache to registers_output_cache. num_threads = + * num_registers_per_sketch. + * + * Input e.g.: + * + * group_labels_thread_cache: + * [ + * g0 + * g0 + * g1 + * ... + * gN + * ] + * Has num_threads rows. + * + * registers_thread_cache: + * [ + * r0_g0, r1_g0, r2_g0, r3_g0, ... , r511_g0 // register values for group 0 + * r0_g0, r1_g0, r2_g0, r3_g0, ... , r511_g0 // register values for group 0 + * r0_g1, r1_g1, r2_g1, r3_g1, ... , r511_g1 // register values for group 1 + * ... + * r0_gN, r1_gN, r2_gN, r3_gN, ... , r511_gN // register values for group N + * ] + * Has num_threads rows, each row is corresponding to + * `group_labels_thread_cache` + * + * registers_output_cache: + * [ + * r0_g0, r1_g0, r2_g0, r3_g0, ... , r511_g0 // register values for group 0 + * r0_g1, r1_g1, r2_g1, r3_g1, ... , r511_g1 // register values for group 1 + * ... + * r0_gN, r1_gN, r2_gN, r3_gN, ... , r511_gN // register values for group N + * ] + * registers_output_cache has num_groups rows. + * + * For each thread, scan from the first register to the last register, find the + * max value in the same group, and then update to registers_output_cache + */ +template +CUDF_KERNEL void merge_sketches_vertically(int64_t num_sketches, + int64_t num_registers_per_sketch, + int* const registers_output_cache, + int const* const registers_thread_cache, + cudf::size_type const* const group_labels_thread_cache) +{ + __shared__ int8_t shared_data[block_size]; + auto const tid = cudf::detail::grid_1d::global_thread_id(); + int shared_idx = tid % block_size; + + // register idx is tid + shared_data[shared_idx] = static_cast(0); + int prev_group = group_labels_thread_cache[0]; + for (auto i = 0; i < num_sketches; i++) { + int curr_group = group_labels_thread_cache[i]; + int8_t curr_reg_v = + static_cast(registers_thread_cache[i * num_registers_per_sketch + tid]); + if (curr_group == prev_group) { + if (curr_reg_v > shared_data[shared_idx]) { shared_data[shared_idx] = curr_reg_v; } + } else { + // meets a new group, store the result for previous group + int64_t result_reg_idx = prev_group * num_registers_per_sketch + tid; + int result_curr_reg_v = registers_output_cache[result_reg_idx]; + if (shared_data[shared_idx] > result_curr_reg_v) { + registers_output_cache[result_reg_idx] = shared_data[shared_idx]; + } + + shared_data[shared_idx] = curr_reg_v; + } + prev_group = curr_group; + } + + // handles the last register in this thread + int64_t reg_idx = prev_group * num_registers_per_sketch + tid; + int curr_reg_v = registers_output_cache[reg_idx]; + if (shared_data[shared_idx] > curr_reg_v) { + registers_output_cache[reg_idx] = shared_data[shared_idx]; + } +} + +/** + * @brief Compact register values, compact 10 registers values + * (each register value is 6 bits) into a long. + * This is consistent with Spark. + * Output: long columns which will be composed into a struct column + * + * Number of threads is num_groups * num_long_cols. + * + * e.g., num_registers_per_sketch is 512(precision is 9): + * Input: + * registers_output_cache: + * [ + * r0_g0, r1_g0, r2_g0, r3_g0, ... , r511_g0 // register values for group 0 + * r0_g1, r1_g1, r2_g1, r3_g1, ... , r511_g1 // register values for group 1 + * ... + * r0_gN, r1_gN, r2_gN, r3_gN, ... , r511_gN // register values for group N + * ] + * Has num_groups rows. + * + * Output: + * 52 long columns + * + * e.g.: r0 to r9 integers are all: 00000000-00000000-00000000-00100001, tailing + * 6 bits: 100-001 Compact to one long is: + * 100001-100001-100001-100001-100001-100001-100001-100001-100001-100001 + */ +CUDF_KERNEL void compact_kernel(int64_t const num_groups, + int64_t const num_registers_per_sketch, + cudf::device_span sketches_output, + // num_groups * num_registers_per_sketch integers + cudf::device_span registers_output_cache) +{ + int64_t const tid = cudf::detail::grid_1d::global_thread_id(); + int64_t const num_long_cols = num_registers_per_sketch / REGISTERS_PER_LONG + 1; + if (tid >= num_groups * num_long_cols) { return; } + + int64_t const group_idx = tid / num_long_cols; + int64_t const long_idx = tid % num_long_cols; + + int64_t const reg_begin_idx = + group_idx * num_registers_per_sketch + long_idx * REGISTERS_PER_LONG; + int64_t num_regs = REGISTERS_PER_LONG; + if (long_idx == num_long_cols - 1) { num_regs = num_registers_per_sketch % REGISTERS_PER_LONG; } + + int64_t ten_registers = 0; + for (auto i = 0; i < num_regs; i++) { + int64_t reg_v = registers_output_cache[reg_begin_idx + i]; + int64_t tmp = reg_v << (REGISTER_VALUE_BITS * i); + ten_registers |= tmp; + } + + sketches_output[long_idx][group_idx] = ten_registers; +} + +std::unique_ptr group_hllpp(cudf::column_view const& input, + int64_t const num_groups, + cudf::device_span group_labels, + int64_t const precision, + rmm::cuda_stream_view stream, + rmm::device_async_resource_ref mr) +{ + int64_t num_registers_per_sketch = 1 << precision; + constexpr int64_t block_size = 256; + constexpr int num_hashs_per_thread = 256; // handles 256 items per thread + int64_t num_threads_partial_kernel = + cudf::util::div_rounding_up_safe(input.size(), num_hashs_per_thread); + + auto sketches_output = rmm::device_uvector( + num_groups * num_registers_per_sketch, stream, cudf::get_current_device_resource_ref()); + + { // add this block to release `registers_thread_cache` and + // `group_labels_thread_cache` + auto registers_thread_cache = + rmm::device_uvector(num_threads_partial_kernel * num_registers_per_sketch, + stream, + cudf::get_current_device_resource_ref()); + auto group_labels_thread_cache = rmm::device_uvector( + num_threads_partial_kernel, stream, cudf::get_current_device_resource_ref()); + + { // add this block to release `hash_col` + // 1. compute all the hashs + auto input_table_view = cudf::table_view{{input}}; + auto hash_col = xxhash64(input_table_view, SEED, stream, mr); + hash_col->set_null_mask(cudf::detail::copy_bitmask(input, stream, mr), input.null_count()); + auto d_hashs = cudf::column_device_view::create(hash_col->view(), stream); + + // 2. execute partial group by + int64_t num_blocks_p1 = + cudf::util::div_rounding_up_safe(num_threads_partial_kernel, block_size); + partial_group_sketches_from_hashs_kernel + <<>>(*d_hashs, + group_labels, + precision, + sketches_output.begin(), + registers_thread_cache.begin(), + group_labels_thread_cache.begin()); + } + // 3. merge the intermidate result + auto num_merge_threads = num_registers_per_sketch; + auto num_merge_blocks = cudf::util::div_rounding_up_safe(num_merge_threads, block_size); + merge_sketches_vertically + <<>>( + num_threads_partial_kernel, // num_sketches + num_registers_per_sketch, + sketches_output.begin(), + registers_thread_cache.begin(), + group_labels_thread_cache.begin()); + } + + // 4. create output columns + auto num_long_cols = num_registers_per_sketch / REGISTERS_PER_LONG + 1; + auto const results_iter = cudf::detail::make_counting_transform_iterator(0, [&](int i) { + return cudf::make_numeric_column( + cudf::data_type{cudf::type_id::INT64}, num_groups, cudf::mask_state::UNALLOCATED, stream, mr); + }); + auto children = + std::vector>(results_iter, results_iter + num_long_cols); + + auto host_results_pointer_iter = + thrust::make_transform_iterator(children.begin(), [](auto const& results_column) { + return results_column->mutable_view().template data(); + }); + auto host_results_pointers = + std::vector(host_results_pointer_iter, host_results_pointer_iter + children.size()); + auto d_results = cudf::detail::make_device_uvector_sync(host_results_pointers, stream, mr); + + auto result = cudf::make_structs_column(num_groups, + std::move(children), + 0, // null count + rmm::device_buffer{}, // null mask + stream); + + // 5. compact sketches + auto num_phase3_threads = num_groups * num_long_cols; + auto num_phase3_blocks = cudf::util::div_rounding_up_safe(num_phase3_threads, block_size); + compact_kernel<<>>( + num_groups, num_registers_per_sketch, d_results, sketches_output); + + return result; +} + +/** + * @brief Partial groups sketches in long columns, similar to + * `partial_group_sketches_from_hashs_kernel` It split longs into segments with + * each has `num_longs_per_threads` elements e.g.: num_registers_per_sketch = + * 512. Each sketch uses 52 (512 / 10 + 1) longs. + * + * Input: + * col_0 col_1 col_51 + * sketch_0: long, long, ..., long + * sketch_1: long, long, ..., long + * sketch_2: long, long, ..., long + * + * num_threads = 52 * div_round_up(num_sketches_input, num_longs_per_threads) + * Each thread scans and merge num_longs_per_threads longs, + * and output the max register value when meets a new group. + * For the last long in a thread, outputs the result into + * `registers_thread_cache`. + * + * By split inputs into segments like `partial_group_sketches_from_hashs_kernel` + * and do partial merge, it will use less memory. Then the kernel + * merge_sketches_vertically can be used to merge the intermidate results: + * registers_output_cache, registers_thread_cache + */ +template +CUDF_KERNEL void partial_group_long_sketches_kernel( + cudf::device_span sketches_input, + int64_t const num_sketches_input, + int64_t const num_threads_per_col, + int64_t const num_registers_per_sketch, + int64_t const num_groups, + cudf::device_span group_labels, + // num_groups * num_registers_per_sketch integers + int* const registers_output_cache, + // num_threads * num_registers_per_sketch integers + int* const registers_thread_cache, + // num_threads integers + cudf::size_type* const group_labels_thread_cache) +{ + auto const tid = cudf::detail::grid_1d::global_thread_id(); + auto const num_long_cols = sketches_input.size(); + if (tid >= num_threads_per_col * num_long_cols) { return; } + + auto const long_idx = tid / num_threads_per_col; + auto const thread_idx_in_cols = tid % num_threads_per_col; + int64_t const* const longs_ptr = sketches_input[long_idx]; + + int* const registers_thread_ptr = + registers_thread_cache + thread_idx_in_cols * num_registers_per_sketch; + + auto const sketch_first = thread_idx_in_cols * num_longs_per_threads; + auto const sketch_end = cuda::std::min(sketch_first + num_longs_per_threads, num_sketches_input); + + int num_regs = REGISTERS_PER_LONG; + if (long_idx == num_long_cols - 1) { num_regs = num_registers_per_sketch % REGISTERS_PER_LONG; } + + for (auto i = 0; i < num_regs; i++) { + cudf::size_type prev_group = group_labels[sketch_first]; + int max_reg_v = 0; + int reg_idx_in_sketch = long_idx * REGISTERS_PER_LONG + i; + for (auto sketch_idx = sketch_first; sketch_idx < sketch_end; sketch_idx++) { + cudf::size_type curr_group = group_labels[sketch_idx]; + int curr_reg_v = get_register_value(longs_ptr[sketch_idx], i); + if (curr_group == prev_group) { + // still in the same group, update the max value + if (curr_reg_v > max_reg_v) { max_reg_v = curr_reg_v; } + } else { + // meets new group, save output for the previous group + int64_t output_idx_prev = num_registers_per_sketch * prev_group + reg_idx_in_sketch; + registers_output_cache[output_idx_prev] = max_reg_v; + + // reset + max_reg_v = curr_reg_v; + } + + if (sketch_idx == sketch_end - 1) { + // last item in the segment + int64_t output_idx_curr = num_registers_per_sketch * curr_group + reg_idx_in_sketch; + if (sketch_idx == num_sketches_input - 1) { + // last segment + registers_output_cache[output_idx_curr] = max_reg_v; + max_reg_v = curr_reg_v; + } else { + if (curr_group != group_labels[sketch_idx + 1]) { + // look the first item in the next segment + registers_output_cache[output_idx_curr] = max_reg_v; + max_reg_v = curr_reg_v; + } + } + } + + prev_group = curr_group; + } + + // For each thread, output current max value + registers_thread_ptr[reg_idx_in_sketch] = max_reg_v; + } + + if (long_idx == 0) { + group_labels_thread_cache[thread_idx_in_cols] = group_labels[sketch_end - 1]; + } +} + +/** + * @brief Merge for struct column. Each long contains 10 + * register values. Merge all rows in the same group. + */ +std::unique_ptr group_merge_hllpp( + cudf::column_view const& hll_input, // struct column + int64_t const num_groups, + cudf::device_span group_labels, + int64_t const precision, + rmm::cuda_stream_view stream, + rmm::device_async_resource_ref mr) +{ + int64_t num_registers_per_sketch = 1 << precision; + int64_t const num_sketches = hll_input.size(); + int64_t const num_long_cols = num_registers_per_sketch / REGISTERS_PER_LONG + 1; + constexpr int64_t num_longs_per_threads = 256; + constexpr int64_t block_size = 256; + + int64_t num_threads_per_col_phase1 = + cudf::util::div_rounding_up_safe(num_sketches, num_longs_per_threads); + int64_t num_threads_phase1 = num_threads_per_col_phase1 * num_long_cols; + int64_t num_blocks = cudf::util::div_rounding_up_safe(num_threads_phase1, block_size); + auto registers_output_cache = rmm::device_uvector( + num_registers_per_sketch * num_groups, stream, cudf::get_current_device_resource_ref()); + { + auto registers_thread_cache = + rmm::device_uvector(num_registers_per_sketch * num_threads_phase1, + stream, + cudf::get_current_device_resource_ref()); + auto group_labels_thread_cache = rmm::device_uvector( + num_threads_per_col_phase1, stream, cudf::get_current_device_resource_ref()); + + cudf::structs_column_view scv(hll_input); + auto const input_iter = cudf::detail::make_counting_transform_iterator( + 0, [&](int i) { return scv.get_sliced_child(i, stream).begin(); }); + auto input_cols = std::vector(input_iter, input_iter + num_long_cols); + auto d_inputs = cudf::detail::make_device_uvector_sync( + input_cols, stream, cudf::get_current_device_resource_ref()); + // 1st kernel: partially group + partial_group_long_sketches_kernel + <<>>(d_inputs, + num_sketches, + num_threads_per_col_phase1, + num_registers_per_sketch, + num_groups, + group_labels, + registers_output_cache.begin(), + registers_thread_cache.begin(), + group_labels_thread_cache.begin()); + auto const num_phase2_threads = num_registers_per_sketch; + auto const num_phase2_blocks = cudf::util::div_rounding_up_safe(num_phase2_threads, block_size); + // 2nd kernel: vertical merge + merge_sketches_vertically + <<>>( + num_threads_per_col_phase1, // num_sketches + num_registers_per_sketch, + registers_output_cache.begin(), + registers_thread_cache.begin(), + group_labels_thread_cache.begin()); + } + + // create output columns + auto const results_iter = cudf::detail::make_counting_transform_iterator(0, [&](int i) { + return cudf::make_numeric_column( + cudf::data_type{cudf::type_id::INT64}, num_groups, cudf::mask_state::UNALLOCATED, stream, mr); + }); + auto results = + std::vector>(results_iter, results_iter + num_long_cols); + + auto host_results_pointer_iter = + thrust::make_transform_iterator(results.begin(), [](auto const& results_column) { + return results_column->mutable_view().template data(); + }); + auto host_results_pointers = + std::vector(host_results_pointer_iter, host_results_pointer_iter + results.size()); + auto d_sketches_output = + cudf::detail::make_device_uvector_sync(host_results_pointers, stream, mr); + + // 3rd kernel: compact + auto num_phase3_threads = num_groups * num_long_cols; + auto num_phase3_blocks = cudf::util::div_rounding_up_safe(num_phase3_threads, block_size); + compact_kernel<<>>( + num_groups, num_registers_per_sketch, d_sketches_output, registers_output_cache); + + return make_structs_column(num_groups, std::move(results), 0, rmm::device_buffer{}); +} + +/** + * @brief Launch only 1 block, uses max 1M(2^18 *sizeof(int)) shared memory. + * For each hash, get a pair: (register index, register value). + * Use shared memory to speedup the fetch max atomic operation. + */ +template +CUDF_KERNEL void reduce_hllpp_kernel(cudf::column_device_view hashs, + cudf::device_span output, + int precision) +{ + __shared__ int32_t shared_data[block_size]; + + auto const tid = cudf::detail::grid_1d::global_thread_id(); + auto const num_hashs = hashs.size(); + uint64_t const num_registers_per_sketch = 1L << precision; + int const idx_shift = 64 - precision; + uint64_t const w_padding = 1ULL << (precision - 1); + + // init tmp data + for (int i = tid; i < num_registers_per_sketch; i += block_size) { + shared_data[i] = 0; + } + __syncthreads(); + + // update max reg value for the reg index + for (int i = tid; i < num_hashs; i += block_size) { + int reg_idx = 0; // init value for null hash + int reg_v = 0; // init value for null hash + if (!hashs.is_null(i)) { + // cast to unsigned, then >> will shift without preserve the sign bit. + uint64_t const hash = static_cast(hashs.element(i)); + reg_idx = hash >> idx_shift; + // get the leading zeros + reg_v = static_cast(cuda::std::countl_zero((hash << precision) | w_padding) + 1ULL); + } + + cuda::atomic_ref register_ref(shared_data[reg_idx]); + register_ref.fetch_max(reg_v, cuda::memory_order_relaxed); + } + __syncthreads(); + + // compact from register values (int array) to long array + // each long holds 10 integers, note reg value < 64 which means the bits from + // 7 to highest are all 0. + if (tid * REGISTERS_PER_LONG < num_registers_per_sketch) { + int start = tid * REGISTERS_PER_LONG; + int end = (tid + 1) * REGISTERS_PER_LONG; + if (end > num_registers_per_sketch) { end = num_registers_per_sketch; } + + int64_t ret = 0; + for (int i = 0; i < end - start; i++) { + int shift = i * REGISTER_VALUE_BITS; + int64_t reg = shared_data[start + i]; + ret |= (reg << shift); + } + + output[tid][0] = ret; + } +} + +std::unique_ptr reduce_hllpp(cudf::column_view const& input, + int64_t const precision, + rmm::cuda_stream_view stream, + rmm::device_async_resource_ref mr) +{ + int64_t num_registers_per_sketch = 1L << precision; + // 1. compute all the hashs + auto input_table_view = cudf::table_view{{input}}; + auto hash_col = xxhash64(input_table_view, SEED, stream, mr); + hash_col->set_null_mask(cudf::detail::copy_bitmask(input, stream, mr), input.null_count()); + auto d_hashs = cudf::column_device_view::create(hash_col->view(), stream); + + // 2. generate long columns, the size of each long column is 1 + auto num_long_cols = num_registers_per_sketch / REGISTERS_PER_LONG + 1; + auto const results_iter = cudf::detail::make_counting_transform_iterator(0, [&](int i) { + return cudf::make_numeric_column(cudf::data_type{cudf::type_id::INT64}, + 1 /**num_groups*/, + cudf::mask_state::UNALLOCATED, + stream, + mr); + }); + auto children = + std::vector>(results_iter, results_iter + num_long_cols); + + auto host_results_pointer_iter = + thrust::make_transform_iterator(children.begin(), [](auto const& results_column) { + return results_column->mutable_view().template data(); + }); + auto host_results_pointers = + std::vector(host_results_pointer_iter, host_results_pointer_iter + children.size()); + auto d_results = cudf::detail::make_device_uvector_sync( + host_results_pointers, stream, cudf::get_current_device_resource_ref()); + + // 2. reduce and generate compacted long values + constexpr int64_t block_size = 256; + // max shared memory is 2^18 * 4 = 1M + auto const shared_mem_size = num_registers_per_sketch * sizeof(int32_t); + reduce_hllpp_kernel + <<<1, block_size, shared_mem_size, stream.value()>>>(*d_hashs, d_results, precision); + + // 3. create struct scalar + auto host_results_view_iter = thrust::make_transform_iterator( + children.begin(), [](auto const& results_column) { return results_column->view(); }); + auto views = + std::vector(host_results_view_iter, host_results_view_iter + num_long_cols); + auto table_view = cudf::table_view{views}; + auto table = cudf::table(table_view); + return std::make_unique(std::move(table), true, stream, mr); +} + +CUDF_KERNEL void reduce_merge_hll_kernel_vertically(cudf::device_span sketch_longs, + cudf::size_type num_sketches, + int num_registers_per_sketch, + int* const output) +{ + auto const tid = cudf::detail::grid_1d::global_thread_id(); + if (tid >= num_registers_per_sketch) { return; } + auto long_idx = tid / REGISTERS_PER_LONG; + auto reg_idx_in_long = tid % REGISTERS_PER_LONG; + int max = 0; + for (auto row_idx = 0; row_idx < num_sketches; row_idx++) { + int reg_v = get_register_value(sketch_longs[long_idx][row_idx], reg_idx_in_long); + if (reg_v > max) { max = reg_v; } + } + output[tid] = max; +} + +std::unique_ptr reduce_merge_hllpp(cudf::column_view const& input, + int64_t const precision, + rmm::cuda_stream_view stream, + rmm::device_async_resource_ref mr) +{ + // create device input + int64_t num_registers_per_sketch = 1 << precision; + auto num_long_cols = num_registers_per_sketch / REGISTERS_PER_LONG + 1; + cudf::structs_column_view scv(input); + auto const input_iter = cudf::detail::make_counting_transform_iterator( + 0, [&](int i) { return scv.get_sliced_child(i, stream).begin(); }); + auto input_cols = std::vector(input_iter, input_iter + num_long_cols); + auto d_inputs = cudf::detail::make_device_uvector_sync( + input_cols, stream, cudf::get_current_device_resource_ref()); + + // create one row output + auto const results_iter = cudf::detail::make_counting_transform_iterator(0, [&](int i) { + return cudf::make_numeric_column(cudf::data_type{cudf::type_id::INT64}, + 1 /** num_rows */, + cudf::mask_state::UNALLOCATED, + stream, + mr); + }); + auto children = + std::vector>(results_iter, results_iter + num_long_cols); + + auto host_results_pointer_iter = + thrust::make_transform_iterator(children.begin(), [](auto const& results_column) { + return results_column->mutable_view().template data(); + }); + auto host_results_pointers = + std::vector(host_results_pointer_iter, host_results_pointer_iter + children.size()); + auto d_results = cudf::detail::make_device_uvector_sync(host_results_pointers, stream, mr); + + // execute merge kernel + auto num_threads = num_registers_per_sketch; + constexpr int64_t block_size = 256; + auto num_blocks = cudf::util::div_rounding_up_safe(num_threads, block_size); + auto output_cache = rmm::device_uvector( + num_registers_per_sketch, stream, cudf::get_current_device_resource_ref()); + reduce_merge_hll_kernel_vertically<<>>( + d_inputs, input.size(), num_registers_per_sketch, output_cache.begin()); + + // compact to longs + auto const num_compact_threads = num_long_cols; + auto const num_compact_blocks = cudf::util::div_rounding_up_safe(num_compact_threads, block_size); + compact_kernel<<>>( + 1 /** num_groups **/, num_registers_per_sketch, d_results, output_cache); + + // create scalar + auto host_results_view_iter = thrust::make_transform_iterator( + children.begin(), [](auto const& results_column) { return results_column->view(); }); + auto views = + std::vector(host_results_view_iter, host_results_view_iter + num_long_cols); + auto table_view = cudf::table_view{views}; + auto table = cudf::table(table_view); + return std::make_unique(std::move(table), true, stream, mr); +} + +struct estimate_fn { + cudf::device_span sketches; + int64_t* out; + int precision; + + __device__ void operator()(cudf::size_type const idx) const + { + auto const num_regs = 1 << precision; + double sum = 0; + int zeroes = 0; + + for (auto reg_idx = 0; reg_idx < num_regs; ++reg_idx) { + // each long contains 10 register values + int long_col_idx = reg_idx / REGISTERS_PER_LONG; + int reg_idx_in_long = reg_idx % REGISTERS_PER_LONG; + int reg = get_register_value(sketches[long_col_idx][idx], reg_idx_in_long); + sum += double{1} / static_cast(1ull << reg); + zeroes += reg == 0; + } + + auto const finalize = cuco::hyperloglog_ns::detail::finalizer(precision); + out[idx] = finalize(sum, zeroes); + } +}; + +} // end anonymous namespace + +std::unique_ptr group_hyper_log_log_plus_plus( + cudf::column_view const& input, + int64_t const num_groups, + cudf::device_span group_labels, + int64_t const precision, + rmm::cuda_stream_view stream, + rmm::device_async_resource_ref mr) +{ + CUDF_EXPECTS(precision >= 4, "HyperLogLogPlusPlus requires precision bigger than 4."); + auto adjust_precision = precision > MAX_PRECISION ? MAX_PRECISION : precision; + return group_hllpp(input, num_groups, group_labels, adjust_precision, stream, mr); +} + +std::unique_ptr group_merge_hyper_log_log_plus_plus( + cudf::column_view const& input, + int64_t const num_groups, + cudf::device_span group_labels, + int64_t const precision, + rmm::cuda_stream_view stream, + rmm::device_async_resource_ref mr) +{ + CUDF_EXPECTS(precision >= 4, "HyperLogLogPlusPlus requires precision bigger than 4."); + CUDF_EXPECTS(input.type().id() == cudf::type_id::STRUCT, + "HyperLogLogPlusPlus buffer type must be a STRUCT of long columns."); + for (auto i = 0; i < input.num_children(); i++) { + CUDF_EXPECTS(input.child(i).type().id() == cudf::type_id::INT64, + "HyperLogLogPlusPlus buffer type must be a STRUCT of long columns."); + } + auto adjust_precision = precision > MAX_PRECISION ? MAX_PRECISION : precision; + auto expected_num_longs = (1 << adjust_precision) / REGISTERS_PER_LONG + 1; + CUDF_EXPECTS(input.num_children() == expected_num_longs, + "The num of long columns in input is incorrect."); + return group_merge_hllpp(input, num_groups, group_labels, adjust_precision, stream, mr); +} + +std::unique_ptr reduce_hyper_log_log_plus_plus(cudf::column_view const& input, + int64_t const precision, + rmm::cuda_stream_view stream, + rmm::device_async_resource_ref mr) +{ + CUDF_EXPECTS(precision >= 4, "HyperLogLogPlusPlus requires precision bigger than 4."); + auto adjust_precision = precision > MAX_PRECISION ? MAX_PRECISION : precision; + return reduce_hllpp(input, adjust_precision, stream, mr); +} + +std::unique_ptr reduce_merge_hyper_log_log_plus_plus( + cudf::column_view const& input, + int64_t const precision, + rmm::cuda_stream_view stream, + rmm::device_async_resource_ref mr) +{ + CUDF_EXPECTS(precision >= 4, "HyperLogLogPlusPlus requires precision bigger than 4."); + CUDF_EXPECTS(input.type().id() == cudf::type_id::STRUCT, + "HyperLogLogPlusPlus buffer type must be a STRUCT of long columns."); + for (auto i = 0; i < input.num_children(); i++) { + CUDF_EXPECTS(input.child(i).type().id() == cudf::type_id::INT64, + "HyperLogLogPlusPlus buffer type must be a STRUCT of long columns."); + } + auto adjust_precision = precision > MAX_PRECISION ? MAX_PRECISION : precision; + auto expected_num_longs = (1 << adjust_precision) / REGISTERS_PER_LONG + 1; + CUDF_EXPECTS(input.num_children() == expected_num_longs, + "The num of long columns in input is incorrect."); + return reduce_merge_hllpp(input, adjust_precision, stream, mr); +} + +std::unique_ptr estimate_from_hll_sketches(cudf::column_view const& input, + int precision, + rmm::cuda_stream_view stream, + rmm::device_async_resource_ref mr) +{ + CUDF_EXPECTS(precision >= 4, "HyperLogLogPlusPlus requires precision bigger than 4."); + CUDF_EXPECTS(input.type().id() == cudf::type_id::STRUCT, + "HyperLogLogPlusPlus buffer type must be a STRUCT of long columns."); + for (auto i = 0; i < input.num_children(); i++) { + CUDF_EXPECTS(input.child(i).type().id() == cudf::type_id::INT64, + "HyperLogLogPlusPlus buffer type must be a STRUCT of long columns."); + } + auto const input_iter = cudf::detail::make_counting_transform_iterator( + 0, [&](int i) { return input.child(i).begin(); }); + auto const h_input_ptrs = + std::vector(input_iter, input_iter + input.num_children()); + auto d_inputs = cudf::detail::make_device_uvector_sync( + h_input_ptrs, stream, cudf::get_current_device_resource_ref()); + auto result = cudf::make_numeric_column( + cudf::data_type{cudf::type_id::INT64}, input.size(), cudf::mask_state::UNALLOCATED, stream, mr); + // evaluate from struct + thrust::for_each_n(rmm::exec_policy_nosync(stream), + thrust::make_counting_iterator(0), + input.size(), + estimate_fn{d_inputs, result->mutable_view().data(), precision}); + return result; +} + +} // namespace spark_rapids_jni diff --git a/src/main/cpp/src/hyper_log_log_plus_plus.hpp b/src/main/cpp/src/hyper_log_log_plus_plus.hpp new file mode 100644 index 000000000..c2f567d6f --- /dev/null +++ b/src/main/cpp/src/hyper_log_log_plus_plus.hpp @@ -0,0 +1,86 @@ +/* + * Copyright (c) 2024-2025, 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 spark_rapids_jni { + +/** + * @brief Compute hash codes for the input, generate HyperLogLogPlusPlus(HLLPP) + * sketches from hash codes, and merge the sketches in the same group. Output is + * a struct column with multiple long columns which is consistent with Spark. + */ +std::unique_ptr group_hyper_log_log_plus_plus( + cudf::column_view const& input, + int64_t const num_groups, + cudf::device_span group_labels, + int64_t const precision, + rmm::cuda_stream_view stream = cudf::get_default_stream(), + rmm::device_async_resource_ref mr = cudf::get_current_device_resource_ref()); + +/** + * @brief Merge HyperLogLogPlusPlus(HLLPP) sketches in the same group. + * Input is a struct column with multiple long columns which is consistent with + * Spark. + */ +std::unique_ptr group_merge_hyper_log_log_plus_plus( + cudf::column_view const& input, + int64_t const num_groups, + cudf::device_span group_labels, + int64_t const precision, + rmm::cuda_stream_view stream = cudf::get_default_stream(), + rmm::device_async_resource_ref mr = cudf::get_current_device_resource_ref()); + +/** + * @brief Compute hash codes for the input, generate HyperLogLogPlusPlus(HLLPP) + * sketches from hash codes, and merge all the sketches into one sketch, output + * is a struct scalar with multiple long values. + */ +std::unique_ptr reduce_hyper_log_log_plus_plus( + cudf::column_view const& input, + int64_t const precision, + rmm::cuda_stream_view stream = cudf::get_default_stream(), + rmm::device_async_resource_ref mr = cudf::get_current_device_resource_ref()); + +/** + * @brief Merge all HyperLogLogPlusPlus(HLLPP) sketches in the input column into one + * sketch. Input is a struct column with multiple long columns which is + * consistent with Spark. Output is a struct scalar with multiple long values. + */ +std::unique_ptr reduce_merge_hyper_log_log_plus_plus( + cudf::column_view const& input, + int64_t const precision, + rmm::cuda_stream_view stream = cudf::get_default_stream(), + rmm::device_async_resource_ref mr = cudf::get_current_device_resource_ref()); + +/** + * @brief Estimate count distinct values from HyperLogLogPlusPlus(HLLPP) sketches. + * Input is a struct column with multiple long columns which is consistent with + * Spark. Output is a long column with all values are not null. Spark returns 0 + * for null values when doing APPROX_COUNT_DISTINCT. + */ +std::unique_ptr estimate_from_hll_sketches( + cudf::column_view const& input, + int precision, + rmm::cuda_stream_view stream = cudf::get_default_stream(), + rmm::device_async_resource_ref mr = cudf::get_current_device_resource_ref()); +} // namespace spark_rapids_jni diff --git a/src/main/cpp/src/hyper_log_log_plus_plus_const.hpp b/src/main/cpp/src/hyper_log_log_plus_plus_const.hpp new file mode 100644 index 000000000..b26c38320 --- /dev/null +++ b/src/main/cpp/src/hyper_log_log_plus_plus_const.hpp @@ -0,0 +1,37 @@ +/* + * Copyright (c) 2025, 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 + +namespace spark_rapids_jni { + +/** + * @brief The number of bits that is required for a HLLPP register value. + * + * This number is determined by the maximum number of leading binary zeros a + * hashcode can produce. This is equal to the number of bits the hashcode + * returns. The current implementation uses a 64-bit hashcode, this means 6-bits + * are (at most) needed to store the number of leading zeros. + */ +constexpr int REGISTER_VALUE_BITS = 6; + +/** + * @brief The number of registers that can be stored in a single long. + * It's 64 / 6 = 10. + */ +constexpr int REGISTERS_PER_LONG = 64 / REGISTER_VALUE_BITS; + +} // namespace spark_rapids_jni diff --git a/src/main/cpp/src/hyper_log_log_plus_plus_host_udf.cu b/src/main/cpp/src/hyper_log_log_plus_plus_host_udf.cu new file mode 100644 index 000000000..339cc2c43 --- /dev/null +++ b/src/main/cpp/src/hyper_log_log_plus_plus_host_udf.cu @@ -0,0 +1,177 @@ +/* + * Copyright (c) 2024-2025, 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 "hyper_log_log_plus_plus.hpp" +#include "hyper_log_log_plus_plus_const.hpp" +#include "hyper_log_log_plus_plus_host_udf.hpp" + +#include +#include + +namespace spark_rapids_jni { + +namespace { + +struct hllpp_groupby_udf : cudf::groupby_host_udf { + hllpp_groupby_udf(int precision_, bool is_merge_) : precision(precision_), is_merge(is_merge_) {} + + /** + * Perform the main groupby computation for HLLPP UDF + */ + [[nodiscard]] std::unique_ptr operator()( + rmm::cuda_stream_view stream, rmm::device_async_resource_ref mr) const override + { + auto const group_values = get_grouped_values(); + if (group_values.size() == 0) { return get_empty_output(stream, mr); } + int num_groups = get_num_groups(); + auto const group_labels = get_group_labels(); + if (is_merge) { + // group by intermidate result, group_values are struct of long columns + return spark_rapids_jni::group_merge_hyper_log_log_plus_plus( + group_values, num_groups, group_labels, precision, stream, mr); + } else { + return spark_rapids_jni::group_hyper_log_log_plus_plus( + group_values, num_groups, group_labels, precision, stream, mr); + } + } + + /** + * @brief Create an empty column when the input is empty. + */ + [[nodiscard]] std::unique_ptr get_empty_output( + rmm::cuda_stream_view stream, rmm::device_async_resource_ref mr) const override + { + int num_registers = 1 << precision; + int num_long_cols = num_registers / REGISTERS_PER_LONG + 1; + auto const results_iter = cudf::detail::make_counting_transform_iterator( + 0, [&](int i) { return cudf::make_empty_column(cudf::data_type{cudf::type_id::INT64}); }); + auto children = + std::vector>(results_iter, results_iter + num_long_cols); + return cudf::make_structs_column(0, + std::move(children), + 0, // null count + rmm::device_buffer{}, // null mask + stream, + mr); + } + + [[nodiscard]] bool is_equal(cudf::host_udf_base const& other) const override + { + auto o = dynamic_cast(&other); + return o != nullptr && o->precision == this->precision && o->is_merge == this->is_merge; + } + + [[nodiscard]] std::size_t do_hash() const override + { + return 31 * (31 * std::hash{}({"hllpp_groupby_udf"}) + precision) + is_merge; + } + + [[nodiscard]] std::unique_ptr clone() const override + { + return std::make_unique(precision, is_merge); + } + + private: + int precision; + bool is_merge; +}; + +struct hllpp_reduct_udf : cudf::reduce_host_udf { + hllpp_reduct_udf(int precision_, bool is_merge_) : precision(precision_), is_merge(is_merge_) {} + + /** + * @brief Create an empty scalar when the input is empty. + */ + std::unique_ptr get_empty_scalar(rmm::cuda_stream_view stream, + rmm::device_async_resource_ref mr) const + { + int num_registers = 1 << precision; + int num_long_cols = num_registers / REGISTERS_PER_LONG + 1; + auto const results_iter = cudf::detail::make_counting_transform_iterator( + 0, [&](int i) { return cudf::make_empty_column(cudf::data_type{cudf::type_id::INT64}); }); + auto children = + std::vector>(results_iter, results_iter + num_long_cols); + auto host_results_view_iter = thrust::make_transform_iterator( + children.begin(), [](auto const& results_column) { return results_column->view(); }); + auto views = std::vector(host_results_view_iter, + host_results_view_iter + num_long_cols); + auto table_view = cudf::table_view{views}; + auto table = cudf::table(table_view); + return std::make_unique(std::move(table), true, stream, mr); + } + + /** + * Perform the main reduce computation for HLLPP UDF + */ + std::unique_ptr operator()( + cudf::column_view const& input, + cudf::data_type, /** output_dtype is useless */ + std::optional>, /** init is useless */ + rmm::cuda_stream_view stream, + rmm::device_async_resource_ref mr) const override + { + if (input.size() == 0) { return get_empty_scalar(stream, mr); } + if (is_merge) { + // reduce intermidate result, input are struct of long columns + return spark_rapids_jni::reduce_merge_hyper_log_log_plus_plus(input, precision, stream, mr); + } else { + return spark_rapids_jni::reduce_hyper_log_log_plus_plus(input, precision, stream, mr); + } + } + + [[nodiscard]] bool is_equal(cudf::host_udf_base const& other) const override + { + auto o = dynamic_cast(&other); + return o != nullptr && o->precision == this->precision && o->is_merge == this->is_merge; + } + + [[nodiscard]] std::size_t do_hash() const override + { + return 31 * (31 * std::hash{}({"hllpp_reduct_udf"}) + precision) + is_merge; + } + + [[nodiscard]] std::unique_ptr clone() const override + { + return std::make_unique(precision, is_merge); + } + + private: + int precision; + bool is_merge; +}; + +} // namespace + +cudf::host_udf_base* create_hllpp_reduction_host_udf(int precision) +{ + return new hllpp_reduct_udf(precision, /*is_merge*/ false); +} + +cudf::host_udf_base* create_hllpp_reduction_merge_host_udf(int precision) +{ + return new hllpp_reduct_udf(precision, /*is_merge*/ true); +} + +cudf::host_udf_base* create_hllpp_groupby_host_udf(int precision) +{ + return new hllpp_groupby_udf(precision, /*is_merge*/ false); +} + +cudf::host_udf_base* create_hllpp_groupby_merge_host_udf(int precision) +{ + return new hllpp_groupby_udf(precision, /*is_merge*/ true); +} + +} // namespace spark_rapids_jni diff --git a/src/main/cpp/src/hyper_log_log_plus_plus_host_udf.hpp b/src/main/cpp/src/hyper_log_log_plus_plus_host_udf.hpp new file mode 100644 index 000000000..08bd24ffc --- /dev/null +++ b/src/main/cpp/src/hyper_log_log_plus_plus_host_udf.hpp @@ -0,0 +1,31 @@ +/* + * Copyright (c) 2024-2025, 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 + +namespace spark_rapids_jni { + +cudf::host_udf_base* create_hllpp_reduction_host_udf(int precision); + +cudf::host_udf_base* create_hllpp_reduction_merge_host_udf(int precision); + +cudf::host_udf_base* create_hllpp_groupby_host_udf(int precision); + +cudf::host_udf_base* create_hllpp_groupby_merge_host_udf(int precision); + +} // namespace spark_rapids_jni diff --git a/src/main/cpp/tests/CMakeLists.txt b/src/main/cpp/tests/CMakeLists.txt index c774d3061..89f456b99 100644 --- a/src/main/cpp/tests/CMakeLists.txt +++ b/src/main/cpp/tests/CMakeLists.txt @@ -1,5 +1,5 @@ #============================================================================= -# Copyright (c) 2022-2024, NVIDIA CORPORATION. +# Copyright (c) 2022-2025, NVIDIA CORPORATION. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. @@ -82,3 +82,6 @@ ConfigureTest(PARSE_URI ConfigureTest(SUBSTRING_INDEX substring_index.cpp) +ConfigureTest(HLLPP + hyper_log_log_plus_plus.cu) + diff --git a/src/main/cpp/tests/hyper_log_log_plus_plus.cu b/src/main/cpp/tests/hyper_log_log_plus_plus.cu new file mode 100644 index 000000000..0e3877405 --- /dev/null +++ b/src/main/cpp/tests/hyper_log_log_plus_plus.cu @@ -0,0 +1,277 @@ +/* + * Copyright (c) 2025, 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 +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include + +#include +#include + +using doubles_col = cudf::test::fixed_width_column_wrapper; +using int64_col = cudf::test::fixed_width_column_wrapper; + +struct HyperLogLogPlusPlusUDFTest : cudf::test::BaseFixture {}; + +namespace { + +/** + * @brief Concatenate struct scalars into a struct column. + * @param scalar_col_ptrs Pointers to the columns in scalars, this span appends the column pointes + * of scalar one by one, the size of the vector is num_scalars * num_longs_in_scalar. + * @param num_scalars Number of struct scalars + * @param num_longs_in_scalar Number of long columns in each struct scalar + */ +CUDF_KERNEL void concat_struct_scalars_to_struct_column_kernel( + cudf::device_span scalar_col_ptrs, + int num_scalars, + int num_longs_in_scalar, + cudf::device_span output) +{ + for (auto col = 0; col < num_longs_in_scalar; ++col) { + for (auto scalar_idx = 0; scalar_idx < num_scalars; ++scalar_idx) { + auto flattened_col_idx = scalar_idx * num_longs_in_scalar + col; + output[col][scalar_idx] = scalar_col_ptrs[flattened_col_idx][0]; + } + } +} + +/** + * @brief Flatten columns in scalars into a vector. + */ +std::vector get_column_ptrs_from_struct_scalars( + std::vector>& scalars, int num_longs_in_scalar) +{ + std::vector col_ptrs(num_longs_in_scalar * scalars.size()); + int idx = 0; + for (auto const& s : scalars) { + auto const struct_scalar_ptr = dynamic_cast(s.get()); + auto const table_view = struct_scalar_ptr->view(); + for (auto const& col_view : table_view) { + col_ptrs[idx++] = col_view.data(); + } + } + return col_ptrs; +} + +/** + * @brief Make a struct column from multiple scalars with checks: each scalar is a struct(long) + */ +std::unique_ptr make_struct_column_from_scalars( + std::vector>& scalars, + int num_longs_in_scalar, + rmm::cuda_stream_view stream, + rmm::device_async_resource_ref mr) +{ + // asserts + for (auto const& s : scalars) { + EXPECT_EQ(s->type().id(), cudf::type_id::STRUCT); + auto const struct_scalar_ptr = dynamic_cast(s.get()); + auto const table_view = struct_scalar_ptr->view(); + EXPECT_EQ(num_longs_in_scalar, table_view.num_columns()); + for (auto const& col_view : table_view) { + EXPECT_EQ(col_view.type().id(), cudf::type_id::INT64); + } + } + + // get column pointers from struct scalars + auto col_ptrs = get_column_ptrs_from_struct_scalars(scalars, num_longs_in_scalar); + auto d_col_ptrs = cudf::detail::make_device_uvector_sync(col_ptrs, stream, mr); + + // create output columns + auto const results_iter = cudf::detail::make_counting_transform_iterator(0, [&](int i) { + return cudf::make_numeric_column(cudf::data_type{cudf::type_id::INT64}, + scalars.size(), // num_rows + cudf::mask_state::UNALLOCATED, + stream, + mr); + }); + auto children = + std::vector>(results_iter, results_iter + num_longs_in_scalar); + auto host_results_pointer_iter = + thrust::make_transform_iterator(children.begin(), [](auto const& results_column) { + return results_column->mutable_view().template data(); + }); + auto host_results_pointers = + std::vector(host_results_pointer_iter, host_results_pointer_iter + children.size()); + auto d_output = cudf::detail::make_device_uvector_sync(host_results_pointers, stream, mr); + + // concatenate struct scalars into a struct column + concat_struct_scalars_to_struct_column_kernel<<<1, 1, 0, stream.value()>>>( + d_col_ptrs, scalars.size(), num_longs_in_scalar, d_output); + + // create struct column + return cudf::make_structs_column(scalars.size(), // num_rows + std::move(children), + 0, // null count + rmm::device_buffer{}, // null mask + stream); +} + +/** + * @brief Make a struct column from a single scalar with checks: each scalar is a struct(long) + */ +std::unique_ptr make_struct_column_from_scalar(std::unique_ptr& scalar, + int num_longs_in_scalar, + rmm::cuda_stream_view stream, + rmm::device_async_resource_ref mr) +{ + std::vector> scalars; + scalars.push_back(std::move(scalar)); + return make_struct_column_from_scalars(scalars, num_longs_in_scalar, stream, mr); +} + +} // namespace + +TEST_F(HyperLogLogPlusPlusUDFTest, Reduction) +{ + // 1. Create data + auto const vals1 = doubles_col{1.0, 2.0, 3.0, 4.0, 5.0}; + auto const vals2 = doubles_col{6.0, 7.0, 8.0, 9.0, 10.0}; + auto stream = cudf::get_default_stream(); + auto mr = cudf::get_current_device_resource_ref(); + + // 2. Execute reduce + // There are pow(2, 9) = 512 registers + // Each register stores the num of leading zeros of hash, the max num is 64, + // 6 bits are enough to store the number, so each register is 6 bits. + // All the registers are compacted into 512 / (64 / 6) + 1 = 52 longs + constexpr int precision = 9; + constexpr int num_longs_in_sketch = 512 / 10 + 1; + auto const reduce_agg = + cudf::make_host_udf_aggregation(std::unique_ptr( + spark_rapids_jni::create_hllpp_reduction_host_udf(precision))); + std::vector> reduced_scalars; + for (size_t i = 0; i < 64; i++) { + if (i % 2 == 0) { + auto reduced = + cudf::reduce(vals1, *reduce_agg, cudf::data_type{cudf::type_id::STRUCT}, stream, mr); + EXPECT_TRUE(reduced->is_valid()); + reduced_scalars.push_back(std::move(reduced)); + } else { + auto reduced = + cudf::reduce(vals2, *reduce_agg, cudf::data_type{cudf::type_id::STRUCT}, stream, mr); + EXPECT_TRUE(reduced->is_valid()); + reduced_scalars.push_back(std::move(reduced)); + } + } + + // 3. Merge all the sketches into one sketch + auto const input_for_merge = + make_struct_column_from_scalars(reduced_scalars, num_longs_in_sketch, stream, mr); + auto const merge_agg = + cudf::make_host_udf_aggregation(std::unique_ptr( + spark_rapids_jni::create_hllpp_reduction_merge_host_udf(precision))); + + auto reduce_merged = + cudf::reduce(*input_for_merge, *merge_agg, cudf::data_type{cudf::type_id::STRUCT}, stream, mr); + EXPECT_TRUE(reduce_merged->is_valid()); + + // 4. Estimate count distinct values from the merged sketch + auto const input_for_estimate = + make_struct_column_from_scalar(reduce_merged, num_longs_in_sketch, stream, mr); + auto const result = + spark_rapids_jni::estimate_from_hll_sketches(*input_for_estimate, precision, stream, mr); + + // 5. check count distinct value + auto const expected = int64_col{10}; + CUDF_TEST_EXPECT_COLUMNS_EQUAL(expected, *result); +} + +TEST_F(HyperLogLogPlusPlusUDFTest, Groupby) +{ + constexpr int precision = 9; + + // 1. Create data + auto const keys = int64_col{1, 2, 3, 1, 2, 3, 1, 2, 3}; + // Each key in (1, 2, 3) maps to three values: (1.0, 2.0, 3.0) + auto const vals1 = doubles_col{1.0, 1.0, 1.0, 2.0, 2.0, 2.0, 3.0, 3.0, 3.0}; + // Each key in (1, 2, 3) maps to three values: (4.0, 5.0, 6.0) + auto const vals2 = doubles_col{4.0, 4.0, 4.0, 5.0, 5.0, 5.0, 6.0, 6.0, 6.0}; + // Each key in (1, 2, 3) maps to three values: (7.0, 8.0, 9.0) + auto const vals3 = doubles_col{7.0, 7.0, 7.0, 8.0, 8.0, 8.0, 9.0, 9.0, 9.0}; + + // 2. Execute groupby + auto agg1 = + cudf::make_host_udf_aggregation(std::unique_ptr( + spark_rapids_jni::create_hllpp_groupby_host_udf(precision))); + auto agg2 = + cudf::make_host_udf_aggregation(std::unique_ptr( + spark_rapids_jni::create_hllpp_groupby_host_udf(precision))); + auto agg3 = + cudf::make_host_udf_aggregation(std::unique_ptr( + spark_rapids_jni::create_hllpp_groupby_host_udf(precision))); + std::vector agg_requests; + agg_requests.emplace_back(); + agg_requests[0].values = vals1; + agg_requests[0].aggregations.push_back(std::move(agg1)); + agg_requests.emplace_back(); + agg_requests[1].values = vals2; + agg_requests[1].aggregations.push_back(std::move(agg2)); + agg_requests.emplace_back(); + agg_requests[2].values = vals3; + agg_requests[2].aggregations.push_back(std::move(agg3)); + cudf::groupby::groupby gb_obj( + cudf::table_view({keys}), cudf::null_policy::INCLUDE, cudf::sorted::NO, {}, {}); + auto const grp_result = gb_obj.aggregate(agg_requests, cudf::test::get_default_stream()); + // each grouped sketches has 3 rows for keys: 1, 2, 3 + auto const& grouped_sketches_for_vals1 = grp_result.second[0].results[0]->view(); + auto const& grouped_sketches_for_vals2 = grp_result.second[1].results[0]->view(); + auto const& grouped_sketches_for_vals3 = grp_result.second[1].results[0]->view(); + + // 3. Execute merge sketches + auto stream = cudf::get_default_stream(); + auto mr = cudf::get_current_device_resource_ref(); + // each result is 3 rows, concat to 9 rows. + auto const sketches = cudf::concatenate( + std::vector{ + grouped_sketches_for_vals1, grouped_sketches_for_vals2, grouped_sketches_for_vals3}, + stream, + mr); + auto merge_agg = + cudf::make_host_udf_aggregation(std::unique_ptr( + spark_rapids_jni::create_hllpp_groupby_merge_host_udf(precision))); + std::vector merge_requests; + merge_requests.emplace_back(); + merge_requests[0].values = sketches->view(); + merge_requests[0].aggregations.push_back(std::move(merge_agg)); + cudf::groupby::groupby gb_obj2( + cudf::table_view({keys}), cudf::null_policy::INCLUDE, cudf::sorted::NO, {}, {}); + auto const grp_result2 = gb_obj2.aggregate(merge_requests, cudf::test::get_default_stream()); + auto const& merged = grp_result2.second[0].results[0]; + + // 4. Estimate + auto const result = spark_rapids_jni::estimate_from_hll_sketches(*merged, precision, stream, mr); + + // 5. Check result + // each key in (1, 2, 3) has 9 distinct values: (1.0, 2.0, ..., 9.0) + // Note: 9 != 6, estimation result is an approximate value, not 100 accurate + auto const expected = int64_col{6, 6, 6}; + CUDF_TEST_EXPECT_COLUMNS_EQUAL(expected, *result); +} diff --git a/src/main/java/com/nvidia/spark/rapids/jni/HyperLogLogPlusPlusHostUDF.java b/src/main/java/com/nvidia/spark/rapids/jni/HyperLogLogPlusPlusHostUDF.java new file mode 100644 index 000000000..5b8456005 --- /dev/null +++ b/src/main/java/com/nvidia/spark/rapids/jni/HyperLogLogPlusPlusHostUDF.java @@ -0,0 +1,130 @@ +/* + * Copyright (c) 2024-2025, 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. + */ + +package com.nvidia.spark.rapids.jni; + +import java.util.Objects; + +import ai.rapids.cudf.ColumnVector; +import ai.rapids.cudf.ColumnView; +import ai.rapids.cudf.HostUDFWrapper; +import ai.rapids.cudf.NativeDepsLoader; + +/** + * HyperLogLogPlusPlus(HLLPP) utility for aggregation, reduction and estimation. One HLLPP sketch is + * composed of several register values. Register value is the number of leading zero bits in + * xxhash64 hash code. xxhash64 hash code is 64 bits, so 6 bits is enough to store the zero number. + * Spark compacts one HLLPP sketch(6 bits register values) into multiple longs, each long stores 10 + * register values. So The sketch values must be a struct column with multiple long columns in it. + * The children num of this Struct is: num_registers_per_sketch / 10 + 1. The value of + * num_registers_per_sketch = pow(2, precision). + */ +public class HyperLogLogPlusPlusHostUDF extends HostUDFWrapper { + static { + NativeDepsLoader.loadNativeDeps(); + } + + public HyperLogLogPlusPlusHostUDF(AggregationType type, int precision) { + this.type = type; + this.precision = precision; + } + + @Override + public long createUDFInstance() { + return createHLLPPHostUDF(type, precision); + } + + @Override + public int computeHashCode() { + return Objects.hash(this.getClass().getName(), type, precision); + } + + @Override + public boolean isEquals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + HyperLogLogPlusPlusHostUDF other = (HyperLogLogPlusPlusHostUDF) o; + return type == other.type && precision == other.precision; + } + + /** + * HyperLogLogPlusPlus(HLLPP) aggregation/reduction types + */ + public enum AggregationType { + + /** + * Compute hash codes for the input, generate HyperLogLogPlusPlus(HLLPP) + * sketches from hash codes, and merge all the sketches into one sketch, output + * is a struct scalar with multiple long values. + */ + Reduction(0), + + /** + * Merge all HyperLogLogPlusPlus(HLLPP) sketches in the input column into one + * sketch. Input is a struct column with multiple long columns which is + * consistent with Spark. Output is a struct scalar with multiple long values. + */ + ReductionMerge(1), + + /** + * Compute hash codes for the input, generate HyperLogLogPlusPlus(HLLPP) + * sketches from hash codes, and merge the sketches in the same group. Output is + * a struct column with multiple long columns which is consistent with Spark. + */ + GroupBy(2), + + /** + * Merge HyperLogLogPlusPlus(HLLPP) sketches in the same group. + * Input is a struct column with multiple long columns which is consistent with + * Spark. + */ + GroupByMerge(3); + + final int nativeId; + + AggregationType(int nativeId) { + this.nativeId = nativeId; + } + } + + /** + * Create a HyperLogLogPlusPlus(HLLPP) host UDF + */ + private static long createHLLPPHostUDF(AggregationType type, int precision) { + return createHLLPPHostUDF(type.nativeId, precision); + } + + /** + * Compute the approximate count distinct values from sketch values. + * The input is sketch values must be a struct column with multiple long columns in it. + * + * @param input The sketch column which is a struct column with multiple long columns in it. + * @param precision The num of bits for HLLPP register addressing. + * @return A INT64 column with each value indicates the approximate count + * distinct value. + */ + public static ColumnVector estimateDistinctValueFromSketches(ColumnView input, int precision) { + return new ColumnVector(estimateDistinctValueFromSketches(input.getNativeView(), precision)); + } + + private static native long createHLLPPHostUDF(int type, int precision); + + private static native long estimateDistinctValueFromSketches(long inputHandle, int precision); + + private AggregationType type; + private int precision; +} + \ No newline at end of file