From fd2d0e2ce934870903860d51676d1a5e994f5d8a Mon Sep 17 00:00:00 2001 From: nvdbaranec <56695930+nvdbaranec@users.noreply.github.com> Date: Wed, 3 Feb 2021 20:01:22 -0600 Subject: [PATCH 1/8] Pack/unpack functionality to convert tables to and from a serialized format. (#7096) Addresses #3793 Depends on https://github.com/rapidsai/cudf/pull/6864 (This affects contiguous_split.cu. For the purposes of this PR, the only changes that are relevant are those that involve the generation of metadata) - `pack()` performs a `contiguous_split()` on the incoming table to arrange the memory into a unified device buffer, and generates a host-side metadata buffer. These are returned in the `packed_columns` struct. - unpack() takes the data stored in the `packed_columns` struct and returns a deserialized `table_view` that points into it. The intent of this functionality is as follows (pseudocode) ``` // serialize-side table_view t; packed_columns p = pack(t); send_over_network(p.gpu_data); send_over_network(p.metadata); // deserialize-side packed_columns p = receive_from_network(); table_view t = unpack(p); ``` This PR also renames `contiguous_split_result` to `packed_table` (which is just a bundled `table_view` and `packed_column`) Authors: - @nvdbaranec Approvers: - Jake Hemstad (@jrhemstad) - Paul Taylor (@trxcllnt) - Mike Wilson (@hyperbolic2346) URL: https://github.com/rapidsai/cudf/pull/7096 --- CHANGELOG.md | 37 +++ cpp/include/cudf/copying.hpp | 112 +++++++- cpp/include/cudf/detail/copy.hpp | 15 +- cpp/src/copying/contiguous_split.cu | 138 ++++++--- cpp/src/copying/pack.cpp | 254 +++++++++++++++++ cpp/tests/CMakeLists.txt | 3 +- cpp/tests/copying/pack_tests.cu | 425 ++++++++++++++++++++++++++++ cpp/tests/copying/split_tests.cpp | 254 ++++++++++++++--- 8 files changed, 1152 insertions(+), 86 deletions(-) create mode 100644 cpp/src/copying/pack.cpp create mode 100644 cpp/tests/copying/pack_tests.cu diff --git a/CHANGELOG.md b/CHANGELOG.md index f712a700044..28d80c33f10 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,3 +1,40 @@ +# cuDF 0.18.0 (Date TBD) + +## New Features +- PR #6856 Add groupby idxmin, idxmax aggregation +- PR #6847 Add a cmake find module for cuFile in JNI code +- PR #6902 Implement `DataFrame.quantile` for `datetime` and `timedelta` data types +- PR #6814 Implement `cudf::reduce` for `decimal32` and `decimal64` (part 1) +- PR #6929 Add `Index.set_names` api +- PR #6907 Add `replace_null` API with `replace_policy` parameter, `fixed_width` column support +- PR #6885 Share `factorize` implementation with Index and cudf module +- PR #6775 Implement cudf.DateOffset for months +- PR #7069 Pack/unpack for serialization/deserialization of table_views +- PR #7039 Support contains() on lists of primitives + +## Improvements + +- PR #6938 Pass numeric scalars of the same dtype through numeric binops +- PR #6275 Update to official libcu++ on Github +- PR #6838 Fix `columns` & `index` handling in dataframe constructor +- PR #6750 Remove **kwargs from string/categorical methods +- PR #6585 Add dictionary support to libcudf groupby functions +- PR #6909 Support reading byte array backed decimal columns from parquet files +- PR #6939 Use simplified `rmm::exec_policy` +- PR #6512 Refactor rolling.cu to reduce compile time +- PR #6982 Disable some pragma unroll statements in thrust `sort.h` +- PR #7051 Verify decimal cast in java package +- PR #7120 Verify window operations on decimal in java package + +## Bug Fixes + +- PR #6884 Correct the sampling range when sampling with replacement +- PR #6903 Add null count test for apply_boolean_mask +- PR #6922 Fix N/A detection for empty fields in CSV reader +- PR #6912 Fix rmm_mode=managed parameter for gtests +- PR #6943 Fix join with nulls not equal performance +- PR #6945 Fix groupby agg/apply behaviour when no key columns are provided +- PR #6942 Fix cudf::merge gtest for dictionary columns # 0.18.0 Please see https://github.com/rapidsai/cudf/releases/tag/branch-0.18-latest for the latest changes to this development branch. diff --git a/cpp/include/cudf/copying.hpp b/cpp/include/cudf/copying.hpp index c63fa62679f..bb44e33f786 100644 --- a/cpp/include/cudf/copying.hpp +++ b/cpp/include/cudf/copying.hpp @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018-2020, NVIDIA CORPORATION. + * Copyright (c) 2018-2021, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -476,23 +476,50 @@ std::vector split(column_view const& input, std::vector */ std::vector split(table_view const& input, std::vector const& splits); +/** + * @brief Column data in a serialized format + * + * @ingroup copy_split + * + * Contains data from an array of columns in two contiguous buffers: one on host, which contains + * table metadata and one on device which contains the table data. + */ +struct packed_columns { + /** + * @brief Host-side metadata buffer used for reconstructing columns via unpack. + * + * @ingroup copy_split + */ + struct metadata { + metadata(std::vector&& v) : data_(std::move(v)) {} + uint8_t const* data() const { return data_.data(); } + size_t size() const { return data_.size(); } + + private: + std::vector data_; + }; + + std::unique_ptr metadata_; + std::unique_ptr gpu_data; +}; + /** * @brief The result(s) of a `contiguous_split` * * @ingroup copy_split * * Each table_view resulting from a split operation performed by contiguous_split, - * will be returned wrapped in a `contiguous_split_result`. The table_view and internal + * will be returned wrapped in a `packed_table`. The table_view and internal * column_views in this struct are not owned by a top level cudf::table or cudf::column. - * The backing memory is instead owned by the `all_data` field and in one + * The backing memory and metadata is instead owned by the `data` field and is in one * contiguous block. * * The user is responsible for assuring that the `table` or any derived table_views do - * not outlive the memory owned by `all_data` + * not outlive the memory owned by `data` */ -struct contiguous_split_result { +struct packed_table { cudf::table_view table; - std::unique_ptr all_data; + packed_columns data; }; /** @@ -502,7 +529,7 @@ struct contiguous_split_result { * @ingroup copy_split * * The memory for the output views is allocated in a single contiguous `rmm::device_buffer` returned - * in the `contiguous_split_result`. There is no top-level owning table. + * in the `packed_table`. There is no top-level owning table. * * The returned views of `input` are constructed from a vector of indices, that indicate * where each split should occur. The `i`th returned `table_view` is sliced as @@ -514,7 +541,7 @@ struct contiguous_split_result { * * @note It is the caller's responsibility to ensure that the returned views * do not outlive the viewed device memory contained in the `all_data` field of the - * returned contiguous_split_result. + * returned packed_table. * * @code{.pseudo} * Example: @@ -536,11 +563,78 @@ struct contiguous_split_result { * @return The set of requested views of `input` indicated by the `splits` and the viewed memory * buffer. */ -std::vector contiguous_split( +std::vector contiguous_split( cudf::table_view const& input, std::vector const& splits, rmm::mr::device_memory_resource* mr = rmm::mr::get_current_device_resource()); +/** + * @brief Deep-copy a `table_view` into a serialized contiguous memory format + * + * The metadata from the `table_view` is copied into a host vector of bytes and the data from the + * `table_view` is copied into a `device_buffer`. Pass the output of this function into + * `cudf::unpack` to deserialize. + * + * @param input View of the table to pack + * @param[in] mr Optional, The resource to use for all returned device allocations + * @return packed_columns A struct containing the serialized metadata and data in contiguous host + * and device memory respectively + */ +packed_columns pack(cudf::table_view const& input, + rmm::mr::device_memory_resource* mr = rmm::mr::get_current_device_resource()); + +/** + * @brief Produce the metadata used for packing a table stored in a contiguous buffer. + * + * The metadata from the `table_view` is copied into a host vector of bytes which can be used to + * construct a `packed_columns` or `packed_table` structure. The caller is responsible for + * guaranteeing that that all of the columns in the table point into `contiguous_buffer`. + * + * @param input View of the table to pack + * @param contgiuous_buffer A contiguous buffer of device memory which contains the data referenced + * by the columns in `table` + * @param buffer_size The size of `contiguous_buffer`. + * @return Vector of bytes representing the metadata used to `unpack` a packed_columns struct. + */ +packed_columns::metadata pack_metadata(table_view const& table, + uint8_t const* contiguous_buffer, + size_t buffer_size); + +/** + * @brief Deserialize the result of `cudf::pack` + * + * Converts the result of a serialized table into a `table_view` that points to the data stored in + * the contiguous device buffer contained in `input`. + * + * It is the caller's responsibility to ensure that the `table_view` in the output does not outlive + * the data in the input. + * + * No new device memory is allocated in this function. + * + * @param input The packed columns to unpack + * @return The unpacked `table_view` + */ +table_view unpack(packed_columns const& input); + +/** + * @brief Deserialize the result of `cudf::pack` + * + * Converts the result of a serialized table into a `table_view` that points to the data stored in + * the contiguous device buffer contained in `gpu_data` using the metadata contained in the host + * buffer `metadata`. + * + * It is the caller's responsibility to ensure that the `table_view` in the output does not outlive + * the data in the input. + * + * No new device memory is allocated in this function. + * + * @param metadata The host-side metadata buffer resulting from the initial pack() call + * @param gpu_data The device-side contiguous buffer storing the data that will be referenced by + * the resulting `table_view` + * @return The unpacked `table_view` + */ +table_view unpack(uint8_t const* metadata, uint8_t const* gpu_data); + /** * @brief Returns a new column, where each element is selected from either @p lhs or * @p rhs based on the value of the corresponding element in @p boolean_mask diff --git a/cpp/include/cudf/detail/copy.hpp b/cpp/include/cudf/detail/copy.hpp index 19d9d81a948..2783bd7729f 100644 --- a/cpp/include/cudf/detail/copy.hpp +++ b/cpp/include/cudf/detail/copy.hpp @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018-2020, NVIDIA CORPORATION. + * Copyright (c) 2018-2021, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -92,13 +92,22 @@ std::unique_ptr shift( * @copydoc cudf::contiguous_split * * @param stream CUDA stream used for device memory operations and kernel launches. - */ -std::vector contiguous_split( + **/ +std::vector contiguous_split( cudf::table_view const& input, std::vector const& splits, rmm::cuda_stream_view stream = rmm::cuda_stream_default, rmm::mr::device_memory_resource* mr = rmm::mr::get_current_device_resource()); +/** + * @copydoc cudf::pack + * + * @param stream Optional CUDA stream on which to execute kernels + **/ +packed_columns pack(cudf::table_view const& input, + rmm::cuda_stream_view stream = rmm::cuda_stream_default, + rmm::mr::device_memory_resource* mr = rmm::mr::get_current_device_resource()); + /** * @copydoc cudf::allocate_like(column_view const&, size_type, mask_allocation_policy, * rmm::mr::device_memory_resource*) diff --git a/cpp/src/copying/contiguous_split.cu b/cpp/src/copying/contiguous_split.cu index 10e2dfbdaeb..e082624b04d 100644 --- a/cpp/src/copying/contiguous_split.cu +++ b/cpp/src/copying/contiguous_split.cu @@ -1,5 +1,5 @@ /* - * Copyright (c) 2019-2020, NVIDIA CORPORATION. + * Copyright (c) 2019-2021, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -17,6 +17,7 @@ #include #include #include +#include #include #include #include @@ -95,8 +96,11 @@ struct dst_buf_info { int dst_offset; // my offset into the per-partition allocation int value_shift; // amount to shift values down by (for offset buffers) int bit_shift; // # of bits to shift right by (for validity buffers) + size_type valid_count; }; +constexpr size_type copy_block_size = 512; + /** * @brief Copy a single buffer of column data, shifting values (for offset columns), * and validity (for validity buffers) as necessary. @@ -123,6 +127,8 @@ struct dst_buf_info { * @param stride Size of the kernel block * @param value_shift Shift incoming 4-byte offset values down by this amount * @param bit_shift Shift incoming data right by this many bits + * @param num_rows Number of rows being copied + * @param valid_count Optional pointer to a value to store count of set bits */ __device__ void copy_buffer(uint8_t* __restrict__ dst, uint8_t* __restrict__ src, @@ -132,10 +138,14 @@ __device__ void copy_buffer(uint8_t* __restrict__ dst, int src_row_index, uint32_t stride, int value_shift, - int bit_shift) + int bit_shift, + int num_rows, + size_type* valid_count) { src += (src_row_index * element_size); + size_type thread_valid_count = 0; + // handle misalignment. read 16 bytes in 4 byte reads. write in a single 16 byte store. const size_t num_bytes = num_elements * element_size; // how many bytes we're misaligned from 4-byte alignment @@ -157,12 +167,25 @@ __device__ void copy_buffer(uint8_t* __restrict__ dst, v.z -= value_shift; v.w -= value_shift; reinterpret_cast(dst)[pos / 16] = v; + if (valid_count) { + thread_valid_count += (__popc(v.x) + __popc(v.y) + __popc(v.z) + __popc(v.w)); + } pos += stride; } // copy trailing bytes if (t == 0) { - size_t remainder = num_bytes < 16 ? num_bytes : 16 + (num_bytes % 16); + size_t remainder; + if (num_bytes < 16) { + remainder = num_bytes; + } else { + size_t last_bracket = (num_bytes / 16) * 16; + remainder = num_bytes - last_bracket; + if (remainder < 4) { + // we had less than 20 bytes for the last possible 16 byte copy, so copy 16 + the extra + remainder += 16; + } + } // if we're performing a value shift (offsets), or a bit shift (validity) the # of bytes and // alignment must be a multiple of 4. value shifting and bit shifting are mututally exclusive @@ -173,15 +196,42 @@ __device__ void copy_buffer(uint8_t* __restrict__ dst, while (remainder) { uint32_t next = remainder > 0 ? (reinterpret_cast(src)[idx + 1] - value_shift) : 0; - reinterpret_cast(dst)[idx] = (v >> bit_shift) | (next << (32 - bit_shift)); + uint32_t val = (v >> bit_shift) | (next << (32 - bit_shift)); + if (valid_count) { thread_valid_count += __popc(val); } + reinterpret_cast(dst)[idx] = val; v = next; idx++; remainder -= 4; } } else { while (remainder) { - int idx = num_bytes - remainder--; - reinterpret_cast(dst)[idx] = reinterpret_cast(src)[idx]; + int idx = num_bytes - remainder--; + uint32_t val = reinterpret_cast(src)[idx]; + if (valid_count) { thread_valid_count += __popc(val); } + reinterpret_cast(dst)[idx] = val; + } + } + } + + if (valid_count) { + if (num_bytes == 0) { + if (!t) { *valid_count = 0; } + } else { + using BlockReduce = cub::BlockReduce; + __shared__ typename BlockReduce::TempStorage temp_storage; + size_type block_valid_count{BlockReduce(temp_storage).Sum(thread_valid_count)}; + if (!t) { + // we may have copied more bits than there are actual rows in the output. + // so we need to subtract off the count of any bits that shouldn't have been + // considered during the copy step. + int max_row = (num_bytes * 8); + int slack_bits = max_row > num_rows ? max_row - num_rows : 0; + auto slack_mask = set_most_significant_bits(slack_bits); + if (slack_mask > 0) { + uint32_t last_word = reinterpret_cast(dst + (num_bytes - 4))[0]; + block_valid_count -= __popc(last_word & slack_mask); + } + *valid_count = block_valid_count; } } } @@ -222,7 +272,9 @@ __global__ void copy_partition(int num_src_bufs, buf_info[buf_index].src_row_index, blockDim.x, buf_info[buf_index].value_shift, - buf_info[buf_index].bit_shift); + buf_info[buf_index].bit_shift, + buf_info[buf_index].num_rows, + buf_info[buf_index].valid_count > 0 ? &buf_info[buf_index].valid_count : nullptr); } // The block of functions below are all related: @@ -611,9 +663,15 @@ BufInfo build_output_columns(InputIter begin, size_type null_count; std::tie(bitmask_ptr, null_count) = [&]() { if (src.nullable()) { - auto const ptr = reinterpret_cast(base_ptr + current_info->dst_offset); + auto const ptr = + current_info->num_elements == 0 + ? nullptr + : reinterpret_cast(base_ptr + current_info->dst_offset); + auto const null_count = current_info->num_elements == 0 + ? 0 + : (current_info->num_rows - current_info->valid_count); ++current_info; - return std::make_pair(ptr, UNKNOWN_NULL_COUNT); + return std::make_pair(ptr, null_count); } return std::make_pair(static_cast(nullptr), 0); }(); @@ -627,6 +685,7 @@ BufInfo build_output_columns(InputIter begin, // children auto children = std::vector{}; children.reserve(src.num_children()); + current_info = build_output_columns( src.child_begin(), src.child_end(), current_info, std::back_inserter(children), base_ptr); @@ -705,10 +764,10 @@ struct size_of_helper { namespace detail { -std::vector contiguous_split(cudf::table_view const& input, - std::vector const& splits, - rmm::cuda_stream_view stream, - rmm::mr::device_memory_resource* mr) +std::vector contiguous_split(cudf::table_view const& input, + std::vector const& splits, + rmm::cuda_stream_view stream, + rmm::mr::device_memory_resource* mr) { if (input.num_columns() == 0) { return {}; } if (splits.size() > 0) { @@ -726,24 +785,26 @@ std::vector contiguous_split(cudf::table_view const& in } } - size_t const num_partitions = splits.size() + 1; + size_t const num_partitions = splits.size() + 1; + size_t const num_root_columns = input.num_columns(); // if inputs are empty, just return num_partitions empty tables if (input.column(0).size() == 0) { - std::vector result; + // build the empty results + std::vector result; result.reserve(num_partitions); - auto iter = thrust::make_counting_iterator(0); std::transform( iter, iter + num_partitions, std::back_inserter(result), [&input](int partition_index) { - return contiguous_split_result{input, std::make_unique()}; + return packed_table{input, + packed_columns{std::make_unique(pack_metadata( + input, static_cast(nullptr), 0)), + std::make_unique()}}; }); return result; } - size_t const num_root_columns = input.num_columns(); - // compute # of source buffers (column data, validity, children), # of partitions // and total # of buffers size_type const num_src_bufs = count_src_bufs(input.begin(), input.end()); @@ -874,7 +935,8 @@ std::vector contiguous_split(cudf::table_view const& in out_row_index, 0, value_shift, - bit_shift}; + bit_shift, + src_info.is_validity ? 1 : 0}; }); // compute total size of each partition @@ -957,39 +1019,47 @@ std::vector contiguous_split(cudf::table_view const& in // copy. 1 block per buffer { - constexpr int block_size = 512; - copy_partition<<>>( + copy_partition<<>>( num_src_bufs, num_partitions, d_src_bufs, d_dst_bufs, d_dst_buf_info); } + // DtoH dst info (to retrieve null counts) + CUDA_TRY(cudaMemcpyAsync( + h_dst_buf_info, d_dst_buf_info, dst_buf_info_size, cudaMemcpyDeviceToHost, stream.value())); + + stream.synchronize(); + // build the output. - std::vector result; + std::vector result; result.reserve(num_partitions); std::vector cols; cols.reserve(num_root_columns); auto cur_dst_buf_info = h_dst_buf_info; for (size_t idx = 0; idx < num_partitions; idx++) { + // traverse the buffers and build the columns. cur_dst_buf_info = build_output_columns( input.begin(), input.end(), cur_dst_buf_info, std::back_inserter(cols), h_dst_bufs[idx]); - result.push_back(contiguous_split_result{ - cudf::table_view{cols}, std::make_unique(std::move(out_buffers[idx]))}); + + // pack the columns + cudf::table_view t{cols}; + result.push_back(packed_table{ + t, + packed_columns{ + std::make_unique(cudf::pack_metadata( + t, reinterpret_cast(out_buffers[idx].data()), out_buffers[idx].size())), + std::make_unique(std::move(out_buffers[idx]))}}); + cols.clear(); } - // we need to synchronize, as the HtoD memcpy before the copy kernel could still be in-flight as - // we exit the function and our stack frame disappears. also of note : we're overlapping - // construction of the output columns on the cpu while the data they point to is still being - // copied. this can be a significant time savings. - stream.synchronize(); - return result; } }; // namespace detail -std::vector contiguous_split(cudf::table_view const& input, - std::vector const& splits, - rmm::mr::device_memory_resource* mr) +std::vector contiguous_split(cudf::table_view const& input, + std::vector const& splits, + rmm::mr::device_memory_resource* mr) { CUDF_FUNC_RANGE(); return cudf::detail::contiguous_split(input, splits, rmm::cuda_stream_default, mr); diff --git a/cpp/src/copying/pack.cpp b/cpp/src/copying/pack.cpp new file mode 100644 index 00000000000..38c95da6dc7 --- /dev/null +++ b/cpp/src/copying/pack.cpp @@ -0,0 +1,254 @@ +/* + * Copyright (c) 2021, 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 + +namespace cudf { +namespace detail { + +namespace { + +/** + * @brief The data that is stored as anonymous bytes in the `packed_columns` metadata + * field. + * + * The metadata field of the `packed_columns` struct is simply an array of these. + * This struct is exposed here because it is needed by both contiguous_split, pack + * and unpack. + */ +struct serialized_column { + serialized_column(data_type _type, + size_type _size, + size_type _null_count, + int64_t _data_offset, + int64_t _null_mask_offset, + size_type _num_children) + : type(_type), + size(_size), + null_count(_null_count), + data_offset(_data_offset), + null_mask_offset(_null_mask_offset), + num_children(_num_children), + pad(0) + { + } + + data_type type; + size_type size; + size_type null_count; + int64_t data_offset; // offset into contiguous data buffer, or -1 if column data is null + int64_t null_mask_offset; // offset into contiguous data buffer, or -1 if column data is null + size_type num_children; + // Explicitly pad to avoid uninitialized padding bits, allowing `serialized_column` to be bit-wise + // comparable + int pad; +}; + +/** + * @brief Deserialize a single column into a column_view + * + * Deserializes a single column (it's children are assumed to be already deserialized) + * non-recursively. + * + * @param serial_column Serialized column information + * @param children Children for the column + * @param base_ptr Base pointer for the entire contiguous buffer from which all columns + * were serialized into + * @return Fully formed column_view + */ +column_view deserialize_column(serialized_column serial_column, + std::vector const& children, + uint8_t const* base_ptr) +{ + auto const data_ptr = + serial_column.data_offset != -1 ? base_ptr + serial_column.data_offset : nullptr; + auto const null_mask_ptr = + serial_column.null_mask_offset != -1 + ? reinterpret_cast(base_ptr + serial_column.null_mask_offset) + : nullptr; + + return column_view(serial_column.type, + serial_column.size, + data_ptr, + null_mask_ptr, + serial_column.null_count, + 0, + children); +} + +/** + * @brief Build and add metadata for a column and all of it's children, recursively + * + * + * @param metadata Output vector of serialized_column metadata + * @param col Column to build metadata for + * @param base_ptr Base pointer for the entire contiguous buffer from which all columns + * were serialized into + * @param data_size Size of the incoming buffer + */ +void build_column_metadata(std::vector& metadata, + column_view const& col, + uint8_t const* base_ptr, + size_t data_size) +{ + uint8_t const* data_ptr = col.size() == 0 || !col.head() ? nullptr : col.head(); + if (data_ptr != nullptr) { + CUDF_EXPECTS(data_ptr >= base_ptr && data_ptr < base_ptr + data_size, + "Encountered column data outside the range of input buffer"); + } + int64_t const data_offset = data_ptr ? data_ptr - base_ptr : -1; + + uint8_t const* null_mask_ptr = col.size() == 0 || !col.nullable() + ? nullptr + : reinterpret_cast(col.null_mask()); + if (null_mask_ptr != nullptr) { + CUDF_EXPECTS(null_mask_ptr >= base_ptr && null_mask_ptr < base_ptr + data_size, + "Encountered column null mask outside the range of input buffer"); + } + int64_t const null_mask_offset = null_mask_ptr ? null_mask_ptr - base_ptr : -1; + + // add metadata + metadata.emplace_back( + col.type(), col.size(), col.null_count(), data_offset, null_mask_offset, col.num_children()); + + std::for_each( + col.child_begin(), col.child_end(), [&metadata, &base_ptr, &data_size](column_view const& col) { + build_column_metadata(metadata, col, base_ptr, data_size); + }); +} + +} // anonymous namespace + +/** + * @copydoc cudf::detail::pack + */ +packed_columns pack(cudf::table_view const& input, + rmm::cuda_stream_view stream, + rmm::mr::device_memory_resource* mr) +{ + // do a contiguous_split with no splits to get the memory for the table + // arranged as we want it + auto contig_split_result = cudf::detail::contiguous_split(input, {}, stream, mr); + return std::move(contig_split_result[0].data); +} + +template +packed_columns::metadata pack_metadata(ColumnIter begin, + ColumnIter end, + uint8_t const* contiguous_buffer, + size_t buffer_size) +{ + std::vector metadata; + + // first metadata entry is a stub indicating how many total (top level) columns + // there are + metadata.emplace_back(data_type{type_id::EMPTY}, + static_cast(std::distance(begin, end)), + UNKNOWN_NULL_COUNT, + -1, + -1, + 0); + + std::for_each(begin, end, [&metadata, &contiguous_buffer, &buffer_size](column_view const& col) { + build_column_metadata(metadata, col, contiguous_buffer, buffer_size); + }); + + // convert to anonymous bytes + std::vector metadata_bytes; + auto const metadata_begin = reinterpret_cast(metadata.data()); + std::copy(metadata_begin, + metadata_begin + (metadata.size() * sizeof(serialized_column)), + std::back_inserter(metadata_bytes)); + + return packed_columns::metadata{std::move(metadata_bytes)}; +} + +/** + * @copydoc cudf::detail::unpack + */ +table_view unpack(uint8_t const* metadata, uint8_t const* gpu_data) +{ + // gpu data can be null if everything is empty but the metadata must always be valid + CUDF_EXPECTS(metadata != nullptr, "Encountered invalid packed column input"); + auto serialized_columns = reinterpret_cast(metadata); + uint8_t const* base_ptr = gpu_data; + // first entry is a stub where size == the total # of top level columns (see pack_metadata above) + auto const num_columns = serialized_columns[0].size; + size_t current_index = 1; + + std::function(size_type)> get_columns; + get_columns = [&serialized_columns, ¤t_index, base_ptr, &get_columns](size_t num_columns) { + std::vector cols; + for (size_t i = 0; i < num_columns; i++) { + auto serial_column = serialized_columns[current_index]; + current_index++; + + std::vector children = get_columns(serial_column.num_children); + + cols.emplace_back(deserialize_column(serial_column, children, base_ptr)); + } + + return cols; + }; + + return table_view{get_columns(num_columns)}; +} + +} // namespace detail + +/** + * @copydoc cudf::pack + */ +packed_columns pack(cudf::table_view const& input, rmm::mr::device_memory_resource* mr) +{ + CUDF_FUNC_RANGE(); + return detail::pack(input, 0, mr); +} + +/** + * @copydoc cudf::pack_metadata + */ +packed_columns::metadata pack_metadata(table_view const& table, + uint8_t const* contiguous_buffer, + size_t buffer_size) +{ + CUDF_FUNC_RANGE(); + return detail::pack_metadata(table.begin(), table.end(), contiguous_buffer, buffer_size); +} + +/** + * @copydoc cudf::unpack + */ +table_view unpack(packed_columns const& input) +{ + CUDF_FUNC_RANGE(); + return detail::unpack(input.metadata_->data(), + reinterpret_cast(input.gpu_data->data())); +} + +/** + * @copydoc cudf::unpack + */ +table_view unpack(uint8_t const* metadata, uint8_t const* gpu_data) +{ + CUDF_FUNC_RANGE(); + return detail::unpack(metadata, gpu_data); +} + +} // namespace cudf diff --git a/cpp/tests/CMakeLists.txt b/cpp/tests/CMakeLists.txt index 8395a3cc1f2..642b81ca657 100644 --- a/cpp/tests/CMakeLists.txt +++ b/cpp/tests/CMakeLists.txt @@ -387,7 +387,8 @@ set(COPYING_TEST_SRC "${CMAKE_CURRENT_SOURCE_DIR}/copying/shift_tests.cpp" "${CMAKE_CURRENT_SOURCE_DIR}/copying/get_value_tests.cpp" "${CMAKE_CURRENT_SOURCE_DIR}/copying/sample_tests.cpp" - "${CMAKE_CURRENT_SOURCE_DIR}/copying/concatenate_tests.cu") + "${CMAKE_CURRENT_SOURCE_DIR}/copying/concatenate_tests.cu" + "${CMAKE_CURRENT_SOURCE_DIR}/copying/pack_tests.cu") ConfigureTest(COPYING_TEST "${COPYING_TEST_SRC}") diff --git a/cpp/tests/copying/pack_tests.cu b/cpp/tests/copying/pack_tests.cu new file mode 100644 index 00000000000..6f5e933134b --- /dev/null +++ b/cpp/tests/copying/pack_tests.cu @@ -0,0 +1,425 @@ +/* + * Copyright (c) 2021, 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 + +namespace cudf { +namespace test { + +struct PackUnpackTest : public BaseFixture { + void run_test(cudf::table_view const& t) + { + // verify pack/unpack works + auto packed = pack(t); + auto unpacked = unpack(packed); + cudf::test::expect_tables_equal(t, unpacked); + + // verify pack_metadata itself works + auto metadata = pack_metadata( + unpacked, reinterpret_cast(packed.gpu_data->data()), packed.gpu_data->size()); + EXPECT_EQ(metadata.size(), packed.metadata_->size()); + EXPECT_EQ( + std::equal(metadata.data(), metadata.data() + metadata.size(), packed.metadata_->data()), + true); + } + void run_test(std::vector const& t) { run_test(cudf::table_view{t}); } +}; + +// clang-format off +TEST_F(PackUnpackTest, SingleColumnFixedWidth) +{ + fixed_width_column_wrapper col1 ({ 1, 2, 3, 4, 5, 6, 7}, + { 1, 1, 1, 0, 1, 0, 1}); + + this->run_test({col1}); +} + +TEST_F(PackUnpackTest, SingleColumnFixedWidthNonNullable) +{ + fixed_width_column_wrapper col1 ({ 1, 2, 3, 4, 5, 6, 7}); + + this->run_test({col1}); +} + +TEST_F(PackUnpackTest, MultiColumnFixedWidth) +{ + fixed_width_column_wrapper col1 ({ 1, 2, 3, 4, 5, 6, 7}, + { 1, 1, 1, 0, 1, 0, 1}); + fixed_width_column_wrapper col2 ({ 7, 8, 6, 5, 4, 3, 2}, + { 1, 0, 1, 1, 1, 1, 1}); + fixed_width_column_wrapper col3 ({ 8, 4, 2, 0, 7, 1, 3}, + { 0, 1, 1, 1, 1, 1, 1}); + + this->run_test({col1, col2, col3}); +} + +TEST_F(PackUnpackTest, MultiColumnWithStrings) +{ + fixed_width_column_wrapper col1 ({ 1, 2, 3, 4, 5, 6, 7}, + { 1, 1, 1, 0, 1, 0, 1}); + strings_column_wrapper col2 ({"Lorem", "ipsum", "dolor", "sit", "amet", "ort", "ral"}, + { 1, 0, 1, 1, 1, 0, 1}); + strings_column_wrapper col3 ({"", "this", "is", "a", "column", "of", "strings"}); + + this->run_test({col1, col2, col3}); +} + +TEST_F(PackUnpackTest, EmptyColumns) +{ + { + auto empty_string = cudf::strings::detail::make_empty_strings_column(); + cudf::table_view src_table({static_cast(*empty_string)}); + this->run_test(src_table); + } + + { + cudf::test::strings_column_wrapper str{"abc"}; + auto empty_string = cudf::empty_like(str); + cudf::table_view src_table({static_cast(*empty_string)}); + this->run_test(src_table); + } + + { + cudf::test::fixed_width_column_wrapper col0; + cudf::test::dictionary_column_wrapper col1; + cudf::test::strings_column_wrapper col2; + cudf::test::lists_column_wrapper col3; + cudf::test::structs_column_wrapper col4({}); + + cudf::table_view src_table({col0, col1, col2, col3, col4}); + this->run_test(src_table); + } +} + +std::vector> generate_lists(bool include_validity) +{ + using LCW = cudf::test::lists_column_wrapper; + + if(include_validity){ + auto valids = cudf::test::make_counting_transform_iterator(0, [](auto i) { return i % 2 == 0; }); + cudf::test::lists_column_wrapper list0{{1, 2, 3}, + {4, 5}, + {6}, + {{7, 8}, valids}, + {9, 10, 11}, + LCW{}, + LCW{}, + {{-1, -2, -3, -4, -5}, valids}, + {{100, -200}, valids}}; + + cudf::test::lists_column_wrapper list1{{{{1, 2, 3}, valids}, {4, 5}}, + {{LCW{}, LCW{}, {7, 8}, LCW{}}, valids}, + {{LCW{6}}}, + {{{7, 8}, {{9, 10, 11}, valids}, LCW{}}, valids}, + {{LCW{}, {-1, -2, -3, -4, -5}}, valids}, + {{LCW{}}}, + {{-10}, {-100, -200}}, + {{-10, -200}, LCW{}, {8, 9}}, + {LCW{8}, LCW{}, LCW{9}, {5, 6}}}; + + std::vector> out; + out.push_back(list0.release()); + out.push_back(list1.release()); + return out; + } + + cudf::test::lists_column_wrapper list0{{1, 2, 3}, + {4, 5}, + {6}, + {7, 8}, + {9, 10, 11}, + LCW{}, + LCW{}, + {-1, -2, -3, -4, -5}, + {-100, -200}}; + + cudf::test::lists_column_wrapper list1{{{1, 2, 3}, {4, 5}}, + {LCW{}, LCW{}, {7, 8}, LCW{}}, + {{LCW{6}}}, + {{7, 8}, {9, 10, 11}, LCW{}}, + {LCW{}, {-1, -2, -3, -4, -5}}, + {{LCW{}}}, + {{-10}, {-100, -200}}, + {{-10, -200}, LCW{}, {8, 9}}, + {LCW{8}, LCW{}, LCW{9}, {5, 6}}}; + + std::vector> out; + out.push_back(list0.release()); + out.push_back(list1.release()); + return out; +} + +std::vector> generate_structs(bool include_validity) +{ + // 1. String "names" column. + std::vector names{ + "Vimes", "Carrot", "Angua", "Cheery", "Detritus", "Slant", "Fred", "Todd", "Kevin"}; + std::vector names_validity{1, 1, 1, 1, 1, 1, 1, 1, 1}; + strings_column_wrapper names_column(names.begin(), names.end()); + + // 2. Numeric "ages" column. + std::vector ages{5, 10, 15, 20, 25, 30, 100, 101, 102}; + std::vector ages_validity = {1, 1, 1, 1, 0, 1, 0, 0, 1}; + auto ages_column = include_validity ? fixed_width_column_wrapper( + ages.begin(), ages.end(), ages_validity.begin()) + : fixed_width_column_wrapper(ages.begin(), ages.end()); + + // 3. Boolean "is_human" column. + std::vector is_human{true, true, false, false, false, false, true, true, true}; + std::vector is_human_validity{1, 1, 1, 0, 1, 1, 1, 1, 0}; + auto is_human_col = include_validity + ? fixed_width_column_wrapper( + is_human.begin(), is_human.end(), is_human_validity.begin()) + : fixed_width_column_wrapper(is_human.begin(), is_human.end()); + + // Assemble struct column. + auto const struct_validity = std::vector{1, 1, 1, 1, 1, 0, 0, 1, 0}; + auto struct_column = + include_validity + ? structs_column_wrapper({names_column, ages_column, is_human_col}, struct_validity.begin()) + : structs_column_wrapper({names_column, ages_column, is_human_col}); + + std::vector> out; + out.push_back(struct_column.release()); + return out; +} + +std::vector> generate_struct_of_list() +{ + // 1. String "names" column. + std::vector names{ + "Vimes", "Carrot", "Angua", "Cheery", "Detritus", "Slant", "Fred", "Todd", "Kevin"}; + std::vector names_validity{1, 1, 1, 1, 1, 1, 1, 1, 1}; + strings_column_wrapper names_column(names.begin(), names.end()); + + // 2. Numeric "ages" column. + std::vector ages{5, 10, 15, 20, 25, 30, 100, 101, 102}; + std::vector ages_validity = {1, 1, 1, 1, 0, 1, 0, 0, 1}; + auto ages_column = + fixed_width_column_wrapper(ages.begin(), ages.end(), ages_validity.begin()); + + // 3. List column + using LCW = cudf::test::lists_column_wrapper; + std::vector list_validity{1, 1, 1, 1, 1, 0, 1, 0, 1}; + lists_column_wrapper list({{{"abc", "d", "edf"}, {"jjj"}}, + {{"dgaer", "-7"}, LCW{}}, + {{LCW{}}}, + {{"qwerty"}, {"ral", "ort", "tal"}, {"five", "six"}}, + {LCW{}, LCW{}, {"eight", "nine"}}, + {{LCW{}}}, + {{"fun"}, {"a", "bc", "def", "ghij", "klmno", "pqrstu"}}, + {{"seven", "zz"}, LCW{}, {"xyzzy"}}, + {{LCW{"negative 3", " ", "cleveland"}}} }, + list_validity.begin()); + + // Assemble struct column. + auto const struct_validity = std::vector{1, 1, 1, 1, 1, 0, 0, 1, 0}; + auto struct_column = + structs_column_wrapper({names_column, ages_column, list}, struct_validity.begin()); + + std::vector> out; + out.push_back(struct_column.release()); + return out; +} + +std::vector> generate_list_of_struct() +{ + // 1. String "names" column. + std::vector names{ + "Vimes", "Carrot", "Angua", "Cheery", "Detritus", "Slant", "Fred", "Todd", "Kevin", + "Abc", "Def", "Xyz", "Five", "Seventeen", "Dol", "Est"}; + std::vector names_validity{1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 0, 1, 1, 1}; + strings_column_wrapper names_column(names.begin(), names.end()); + + // 2. Numeric "ages" column. + std::vector ages{5, 10, 15, 20, 25, 30, 100, 101, 102, -1, -2, -3, -4, -5, -6, -7}; + std::vector ages_validity = {1, 1, 1, 1, 0, 1, 0, 0, 1, 0, 0, 0, 0, 1, 1, 1}; + auto ages_column = + fixed_width_column_wrapper(ages.begin(), ages.end(), ages_validity.begin()); + + // Assemble struct column. + auto const struct_validity = std::vector{1, 1, 1, 1, 1, 0, 0, 1, 0, 1, 1, 1, 1, 1, 1, 1}; + auto struct_column = + structs_column_wrapper({names_column, ages_column}, struct_validity.begin()); + + + // 3. List column + std::vector list_validity{1, 1, 1, 1, 1, 0, 1, 0, 1}; + + cudf::test::fixed_width_column_wrapper offsets{0, 1, 4, 5, 7, 7, 10, 13, 14, 16}; + auto list = cudf::make_lists_column(9, offsets.release(), struct_column.release(), + 2, cudf::test::detail::make_null_mask(list_validity.begin(), list_validity.begin() + 9)); + + std::vector> out; + out.push_back(std::move(list)); + return out; +} + +TEST_F(PackUnpackTest, Lists) +{ + // lists + { + auto cols = generate_lists(false); + std::vector col_views; + std::transform(cols.begin(), cols.end(), std::back_inserter(col_views), [](std::unique_ptr const& col){ + return static_cast(*col); + }); + cudf::table_view src_table(col_views); + this->run_test(src_table); + } + + // lists with validity + { + auto cols = generate_lists(true); + std::vector col_views; + std::transform(cols.begin(), cols.end(), std::back_inserter(col_views), [](std::unique_ptr const& col){ + return static_cast(*col); + }); + cudf::table_view src_table(col_views); + this->run_test(src_table); + } +} + +TEST_F(PackUnpackTest, Structs) +{ + // structs + { + auto cols = generate_structs(false); + std::vector col_views; + std::transform(cols.begin(), cols.end(), std::back_inserter(col_views), [](std::unique_ptr const& col){ + return static_cast(*col); + }); + cudf::table_view src_table(col_views); + this->run_test(src_table); + } + + // structs with validity + { + auto cols = generate_structs(true); + std::vector col_views; + std::transform(cols.begin(), cols.end(), std::back_inserter(col_views), [](std::unique_ptr const& col){ + return static_cast(*col); + }); + cudf::table_view src_table(col_views); + this->run_test(src_table); + } +} + +TEST_F(PackUnpackTest, NestedTypes) +{ + // build one big table containing, lists, structs, structs, list + std::vector col_views; + + auto lists = generate_lists(true); + std::transform(lists.begin(), lists.end(), std::back_inserter(col_views), [](std::unique_ptr const& col){ + return static_cast(*col); + }); + + auto structs = generate_structs(true); + std::transform(structs.begin(), structs.end(), std::back_inserter(col_views), [](std::unique_ptr const& col){ + return static_cast(*col); + }); + + auto struct_of_list = generate_struct_of_list(); + std::transform(struct_of_list.begin(), struct_of_list.end(), std::back_inserter(col_views), [](std::unique_ptr const& col){ + return static_cast(*col); + }); + + auto list_of_struct = generate_list_of_struct(); + std::transform(list_of_struct.begin(), list_of_struct.end(), std::back_inserter(col_views), [](std::unique_ptr const& col){ + return static_cast(*col); + }); + + cudf::table_view src_table(col_views); + this->run_test(src_table); +} + +TEST_F(PackUnpackTest, NestedEmpty) +{ + // this produces an empty strings column with no children, + // nested inside a list + { + auto empty_string = cudf::strings::detail::make_empty_strings_column(); + auto offsets = cudf::test::fixed_width_column_wrapper({0, 0}); + auto list = cudf::make_lists_column( + 1, offsets.release(), std::move(empty_string), 0, rmm::device_buffer{0}); + + cudf::table_view src_table({static_cast(*list)}); + this->run_test(src_table); + } + + // this produces an empty strings column with children that have no data, + // nested inside a list + { + cudf::test::strings_column_wrapper str{"abc"}; + auto empty_string = cudf::empty_like(str); + auto offsets = cudf::test::fixed_width_column_wrapper({0, 0}); + auto list = cudf::make_lists_column( + 1, offsets.release(), std::move(empty_string), 0, rmm::device_buffer{0}); + + cudf::table_view src_table({static_cast(*list)}); + this->run_test(src_table); + } + + // this produces an empty lists column with children that have no data, + // nested inside a list + { + cudf::test::lists_column_wrapper listw{{1.0f, 2.0f}, {3.0f, 4.0f}}; + auto empty_list = cudf::empty_like(listw); + auto offsets = cudf::test::fixed_width_column_wrapper({0, 0}); + auto list = cudf::make_lists_column( + 1, offsets.release(), std::move(empty_list), 0, rmm::device_buffer{0}); + + cudf::table_view src_table({static_cast(*list)}); + this->run_test(src_table); + } + + // this produces an empty lists column with children that have no data, + // nested inside a list + { + cudf::test::lists_column_wrapper listw{{1.0f, 2.0f}, {3.0f, 4.0f}}; + auto empty_list = cudf::empty_like(listw); + auto offsets = cudf::test::fixed_width_column_wrapper({0, 0}); + auto list = cudf::make_lists_column( + 1, offsets.release(), std::move(empty_list), 0, rmm::device_buffer{0}); + + cudf::table_view src_table({static_cast(*list)}); + this->run_test(src_table); + } + + // this produces an empty struct column with children that have no data, + // nested inside a list + { + cudf::test::fixed_width_column_wrapper ints{0, 1, 2, 3, 4}; + cudf::test::fixed_width_column_wrapper floats{4, 3, 2, 1, 0}; + auto struct_column = cudf::test::structs_column_wrapper({ints, floats}); + auto empty_struct = cudf::empty_like(struct_column); + auto offsets = cudf::test::fixed_width_column_wrapper({0, 0}); + auto list = cudf::make_lists_column( + 1, offsets.release(), std::move(empty_struct), 0, rmm::device_buffer{0}); + + cudf::table_view src_table({static_cast(*list)}); + this->run_test(src_table); + } +} +// clang-format on + +} // namespace test +} // namespace cudf \ No newline at end of file diff --git a/cpp/tests/copying/split_tests.cpp b/cpp/tests/copying/split_tests.cpp index 3d9a4d50ccc..58e77456869 100644 --- a/cpp/tests/copying/split_tests.cpp +++ b/cpp/tests/copying/split_tests.cpp @@ -1,5 +1,5 @@ /* - * Copyright (c) 2019, NVIDIA CORPORATION. + * Copyright (c) 2019-2021, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -211,26 +211,52 @@ TYPED_TEST(SplitTest, SplitEndToSize) // common functions for testing split/contiguous_split template -void split_long_column(SplitFunc Split, CompareFunc Compare) +void split_custom_column(SplitFunc Split, + CompareFunc Compare, + int size, + std::vector const& splits, + bool include_validity) { // the intent here is to stress the various boundary conditions in contiguous_split - // especially the validity copying code. cudf::size_type start = 0; - cudf::size_type size = 10002; srand(824); - auto rvalids = cudf::test::make_counting_transform_iterator(start, [](auto i) { - return static_cast(rand()) / static_cast(RAND_MAX) < 0.5f ? 0 : 1; + + std::vector base_strings( + {"banana", "pear", "apple", "pecans", "vanilla", "cat", "mouse", "green"}); + auto string_randomizer = thrust::make_transform_iterator( + thrust::make_counting_iterator(0), + [&base_strings](cudf::size_type i) { return base_strings[rand() % base_strings.size()]; }); + + auto rvalids = cudf::test::make_counting_transform_iterator(start, [include_validity](auto i) { + return include_validity + ? (static_cast(rand()) / static_cast(RAND_MAX) < 0.5f ? 0 : 1) + : 0; }); + std::vector valids{rvalids, rvalids + size}; cudf::test::fixed_width_column_wrapper col = create_fixed_columns(start, size, valids.begin()); - std::vector splits{ - 2, 16, 31, 35, 64, 97, 158, 190, 638, 899, 900, 901, 996, 4200, 7131, 8111}; - std::vector> expected = - create_expected_columns_for_splits(splits, size, valids); - auto result = Split(col, splits); + std::vector valids2{rvalids, rvalids + size}; + std::vector strings(string_randomizer, string_randomizer + size); + cudf::test::strings_column_wrapper col2(strings.begin(), strings.end(), valids2.begin()); + + std::vector expected; + std::vector> expected_fixed = + create_expected_columns_for_splits(splits, size, valids); + std::vector expected_strings = + create_expected_string_columns_for_splits(strings, splits, valids2); + std::transform(thrust::make_counting_iterator(static_cast(0)), + thrust::make_counting_iterator(expected_fixed.size()), + std::back_inserter(expected), + [&expected_fixed, &expected_strings](size_t i) { + return cudf::table_view({expected_fixed[i], expected_strings[i]}); + }); + + cudf::table_view tbl({col, col2}); + auto result = Split(tbl, splits); EXPECT_EQ(expected.size(), result.size()); @@ -241,13 +267,37 @@ void split_long_column(SplitFunc Split, CompareFunc Compare) TYPED_TEST(SplitTest, LongColumn) { - split_long_column( - [](cudf::column_view const& t, std::vector const& splits) { + split_custom_column( + [](cudf::table_view const& t, std::vector const& splits) { return cudf::split(t, splits); }, - [](cudf::column_view const& expected, cudf::column_view const& result) { - CUDF_TEST_EXPECT_COLUMNS_EQUIVALENT(expected, result); - }); + [](cudf::table_view const& expected, cudf::table_view const& result) { + std::for_each(thrust::make_counting_iterator(0), + thrust::make_counting_iterator(expected.num_columns()), + [&expected, &result](cudf::size_type i) { + CUDF_TEST_EXPECT_COLUMNS_EQUIVALENT(expected.column(i), result.column(i)); + }); + }, + 10002, + std::vector{ + 2, 16, 31, 35, 64, 97, 158, 190, 638, 899, 900, 901, 996, 4200, 7131, 8111}, + true); + + split_custom_column( + [](cudf::table_view const& t, std::vector const& splits) { + return cudf::split(t, splits); + }, + [](cudf::table_view const& expected, cudf::table_view const& result) { + std::for_each(thrust::make_counting_iterator(0), + thrust::make_counting_iterator(expected.num_columns()), + [&expected, &result](cudf::size_type i) { + CUDF_TEST_EXPECT_COLUMNS_EQUIVALENT(expected.column(i), result.column(i)); + }); + }, + 10002, + std::vector{ + 2, 16, 31, 35, 64, 97, 158, 190, 638, 899, 900, 901, 996, 4200, 7131, 8111}, + false); } struct SplitStringTest : public SplitTest { @@ -1122,14 +1172,144 @@ TYPED_TEST_CASE(ContiguousSplitTest, FixedWidthTypesWithoutChrono); TYPED_TEST(ContiguousSplitTest, LongColumn) { - split_long_column( - [](cudf::column_view const& c, std::vector const& splits) { - cudf::table_view t({c}); + split_custom_column( + [](cudf::table_view const& t, std::vector const& splits) { return cudf::contiguous_split(t, splits); }, - [](cudf::column_view const& expected, cudf::contiguous_split_result const& result) { - CUDF_TEST_EXPECT_COLUMNS_EQUIVALENT(expected, result.table.column(0)); - }); + [](cudf::table_view const& expected, cudf::packed_table const& result) { + std::for_each(thrust::make_counting_iterator(0), + thrust::make_counting_iterator(expected.num_columns()), + [&expected, &result](cudf::size_type i) { + CUDF_TEST_EXPECT_COLUMNS_EQUIVALENT(expected.column(i), + result.table.column(i)); + }); + }, + 10002, + std::vector{ + 2, 16, 31, 35, 64, 97, 158, 190, 638, 899, 900, 901, 996, 4200, 7131, 8111}, + true); + + split_custom_column( + [](cudf::table_view const& t, std::vector const& splits) { + return cudf::contiguous_split(t, splits); + }, + [](cudf::table_view const& expected, cudf::packed_table const& result) { + std::for_each(thrust::make_counting_iterator(0), + thrust::make_counting_iterator(expected.num_columns()), + [&expected, &result](cudf::size_type i) { + CUDF_TEST_EXPECT_COLUMNS_EQUIVALENT(expected.column(i), + result.table.column(i)); + }); + }, + 10002, + std::vector{ + 2, 16, 31, 35, 64, 97, 158, 190, 638, 899, 900, 901, 996, 4200, 7131, 8111}, + false); +} + +TYPED_TEST(ContiguousSplitTest, LongColumnBigSplits) +{ + split_custom_column( + [](cudf::table_view const& t, std::vector const& splits) { + return cudf::contiguous_split(t, splits); + }, + [](cudf::table_view const& expected, cudf::packed_table const& result) { + std::for_each(thrust::make_counting_iterator(0), + thrust::make_counting_iterator(expected.num_columns()), + [&expected, &result](cudf::size_type i) { + CUDF_TEST_EXPECT_COLUMNS_EQUIVALENT(expected.column(i), + result.table.column(i)); + }); + }, + 10007, + std::vector{0, 3613, 7777, 10005, 10007}, + true); + + split_custom_column( + [](cudf::table_view const& t, std::vector const& splits) { + return cudf::contiguous_split(t, splits); + }, + [](cudf::table_view const& expected, cudf::packed_table const& result) { + std::for_each(thrust::make_counting_iterator(0), + thrust::make_counting_iterator(expected.num_columns()), + [&expected, &result](cudf::size_type i) { + CUDF_TEST_EXPECT_COLUMNS_EQUIVALENT(expected.column(i), + result.table.column(i)); + }); + }, + 10007, + std::vector{0, 3613, 7777, 10005, 10007}, + false); +} + +// this is a useful test but a little too expensive to run all the time +/* +TYPED_TEST(ContiguousSplitTest, LongColumnTinySplits) +{ + std::vector splits(thrust::make_counting_iterator(0), +thrust::make_counting_iterator(10000)); + + split_custom_column( + [](cudf::table_view const& t, std::vector const& splits) { + return cudf::contiguous_split(t, splits); + }, + [](cudf::table_view const& expected, cudf::packed_table const& result) { + std::for_each(thrust::make_counting_iterator(0), + thrust::make_counting_iterator(expected.num_columns()), + [&expected, &result](cudf::size_type i){ + + // printf("col: "); + // cudf::test::print(result.table.column(i)); + // printf("\n"); + CUDF_TEST_EXPECT_COLUMNS_EQUIVALENT(expected.column(i), result.table.column(i)); + }); + }, + 10002, + splits + ); +} +*/ +struct ContiguousSplitUntypedTest : public cudf::test::BaseFixture { +}; + +TEST_F(ContiguousSplitUntypedTest, ProgressiveSizes) +{ + constexpr int col_size = 256; + + // stress test copying a wide amount of bytes. + for (int idx = 0; idx < col_size; idx++) { + split_custom_column( + [](cudf::table_view const& t, std::vector const& splits) { + return cudf::contiguous_split(t, splits); + }, + [](cudf::table_view const& expected, cudf::packed_table const& result) { + std::for_each(thrust::make_counting_iterator(0), + thrust::make_counting_iterator(expected.num_columns()), + [&expected, &result](cudf::size_type i) { + CUDF_TEST_EXPECT_COLUMNS_EQUIVALENT(expected.column(i), + result.table.column(i)); + }); + }, + col_size, + std::vector{idx}, + true); + + split_custom_column( + [](cudf::table_view const& t, std::vector const& splits) { + return cudf::contiguous_split(t, splits); + }, + [](cudf::table_view const& expected, cudf::packed_table const& result) { + std::for_each(thrust::make_counting_iterator(0), + thrust::make_counting_iterator(expected.num_columns()), + [&expected, &result](cudf::size_type i) { + CUDF_TEST_EXPECT_COLUMNS_EQUIVALENT(expected.column(i), + result.table.column(i)); + }); + }, + col_size, + std::vector{idx}, + false); + } } // contiguous split with strings @@ -1142,7 +1322,7 @@ TEST_F(ContiguousSplitStringTableTest, StringWithInvalids) [](cudf::table_view const& t, std::vector const& splits) { return cudf::contiguous_split(t, splits); }, - [](cudf::table_view const& expected, cudf::contiguous_split_result const& result) { + [](cudf::table_view const& expected, cudf::packed_table const& result) { CUDF_TEST_EXPECT_TABLES_EQUAL(expected, result.table); }); } @@ -1164,7 +1344,7 @@ TEST_F(ContiguousSplitStringTableTest, EmptyInputColumn) { std::vector splits; auto result = cudf::contiguous_split(src_table, splits); - CUDF_EXPECTS(result.size() == 1, "Incorrect returned contiguous_split_result size!"); + CUDF_EXPECTS(result.size() == 1, "Incorrect returned contiguous_split result size!"); CUDF_TEST_EXPECT_TABLES_EQUIVALENT(src_table, result[0].table); } @@ -1172,7 +1352,7 @@ TEST_F(ContiguousSplitStringTableTest, EmptyInputColumn) { std::vector splits{0, 0, 0, 0}; auto result = cudf::contiguous_split(src_table, splits); - CUDF_EXPECTS(result.size() == 5, "Incorrect returned contiguous_split_result size!"); + CUDF_EXPECTS(result.size() == 5, "Incorrect returned contiguous_split result size!"); for (size_t idx = 0; idx < result.size(); idx++) { CUDF_TEST_EXPECT_TABLES_EQUIVALENT(src_table, result[idx].table); @@ -1186,9 +1366,7 @@ TEST_F(ContiguousSplitStringTableTest, EmptyOutputColumn) [](cudf::table_view const& t, std::vector const& splits) { return cudf::contiguous_split(t, splits); }, - [](cudf::contiguous_split_result const& t, int num_cols) { - EXPECT_EQ(t.table.num_columns(), num_cols); - }); + [](cudf::packed_table const& t, int num_cols) { EXPECT_EQ(t.table.num_columns(), num_cols); }); } TEST_F(ContiguousSplitStringTableTest, NullStringColumn) @@ -1197,7 +1375,7 @@ TEST_F(ContiguousSplitStringTableTest, NullStringColumn) [](cudf::table_view const& t, std::vector const& splits) { return cudf::contiguous_split(t, splits); }, - [](cudf::table const& expected, cudf::contiguous_split_result const& result) { + [](cudf::table const& expected, cudf::packed_table const& result) { CUDF_TEST_EXPECT_TABLES_EQUAL(expected.view(), result.table); }); } @@ -1214,7 +1392,7 @@ TYPED_TEST(ContiguousSplitTableTest, SplitEndLessThanSize) [](cudf::table_view const& t, std::vector const& splits) { return cudf::contiguous_split(t, splits); }, - [](cudf::table_view const& expected, cudf::contiguous_split_result const& result) { + [](cudf::table_view const& expected, cudf::packed_table const& result) { CUDF_TEST_EXPECT_TABLES_EQUAL(expected, result.table); }); } @@ -1225,7 +1403,7 @@ TYPED_TEST(ContiguousSplitTableTest, SplitEndToSize) [](cudf::table_view const& t, std::vector const& splits) { return cudf::contiguous_split(t, splits); }, - [](cudf::table_view const& expected, cudf::contiguous_split_result const& result) { + [](cudf::table_view const& expected, cudf::packed_table const& result) { CUDF_TEST_EXPECT_TABLES_EQUAL(expected, result.table); }); } @@ -1274,9 +1452,7 @@ TEST_F(ContiguousSplitTableCornerCases, EmptyOutputColumn) [](cudf::table_view const& t, std::vector const& splits) { return cudf::contiguous_split(t, splits); }, - [](cudf::contiguous_split_result const& t, int num_cols) { - EXPECT_EQ(t.table.num_columns(), num_cols); - }); + [](cudf::packed_table const& t, int num_cols) { EXPECT_EQ(t.table.num_columns(), num_cols); }); } TEST_F(ContiguousSplitTableCornerCases, MixedColumnTypes) @@ -1469,7 +1645,7 @@ TEST_F(ContiguousSplitNestedTypesTest, Lists) cudf::table_view t({c}); return cudf::contiguous_split(t, splits); }, - [](cudf::column_view const& expected, cudf::contiguous_split_result const& result) { + [](cudf::column_view const& expected, cudf::packed_table const& result) { CUDF_TEST_EXPECT_COLUMNS_EQUIVALENT(expected, result.table.column(0)); }); } @@ -1481,7 +1657,7 @@ TEST_F(ContiguousSplitNestedTypesTest, ListsWithNulls) cudf::table_view t({c}); return cudf::contiguous_split(t, splits); }, - [](cudf::column_view const& expected, cudf::contiguous_split_result const& result) { + [](cudf::column_view const& expected, cudf::packed_table const& result) { CUDF_TEST_EXPECT_COLUMNS_EQUIVALENT(expected, result.table.column(0)); }); } @@ -1494,7 +1670,7 @@ TEST_F(ContiguousSplitNestedTypesTest, Structs) cudf::table_view t({c}); return cudf::contiguous_split(t, splits); }, - [](cudf::column_view const& expected, cudf::contiguous_split_result const& result) { + [](cudf::column_view const& expected, cudf::packed_table const& result) { CUDF_TEST_EXPECT_COLUMNS_EQUIVALENT(expected, result.table.column(0)); }); } @@ -1507,7 +1683,7 @@ TEST_F(ContiguousSplitNestedTypesTest, StructsWithNulls) cudf::table_view t({c}); return cudf::contiguous_split(t, splits); }, - [](cudf::column_view const& expected, cudf::contiguous_split_result const& result) { + [](cudf::column_view const& expected, cudf::packed_table const& result) { CUDF_TEST_EXPECT_COLUMNS_EQUIVALENT(expected, result.table.column(0)); }); } @@ -1519,7 +1695,7 @@ TEST_F(ContiguousSplitNestedTypesTest, StructsNoChildren) cudf::table_view t({c}); return cudf::contiguous_split(t, splits); }, - [](cudf::column_view const& expected, cudf::contiguous_split_result const& result) { + [](cudf::column_view const& expected, cudf::packed_table const& result) { CUDF_TEST_EXPECT_COLUMNS_EQUIVALENT(expected, result.table.column(0)); }); } @@ -1531,7 +1707,7 @@ TEST_F(ContiguousSplitNestedTypesTest, StructsOfList) cudf::table_view t({c}); return cudf::contiguous_split(t, splits); }, - [](cudf::column_view const& expected, cudf::contiguous_split_result const& result) { + [](cudf::column_view const& expected, cudf::packed_table const& result) { CUDF_TEST_EXPECT_COLUMNS_EQUIVALENT(expected, result.table.column(0)); }); } From fd38b4c06a19bd6b50b7337e126b9cd6b585dfe3 Mon Sep 17 00:00:00 2001 From: MithunR Date: Wed, 3 Feb 2021 20:48:16 -0800 Subject: [PATCH 2/8] Move lists utility function definition out of header (#7266) Fixes #7265. `cudf::detail::get_num_child_rows()` is currently defined in `cudf/lists/detail/utilities.cuh`. The build pipelines for #7189 are fine, but there seem to be build failures in dependent projects such as `spark-rapids`: ``` [2021-01-31T08:12:10.611Z] /.../workspace/spark/cudf18_nightly/cpp/include/cudf/lists/detail/utilities.cuh:31:18: error: 'cudf::size_type cudf::detail::get_num_child_rows(const cudf::column_view&, rmm::cuda_stream_view)' defined but not used [-Werror=unused-function] [2021-01-31T08:12:10.611Z] static cudf::size_type get_num_child_rows(cudf::column_view const& list_offsets, [2021-01-31T08:12:10.611Z] ^~~~~~~~~~~~~~~~~~ [2021-01-31T08:12:11.981Z] cc1plus: all warnings being treated as errors [2021-01-31T08:12:12.238Z] make[2]: *** [CMakeFiles/cudf_hash.dir/build.make:82: CMakeFiles/cudf_hash.dir/src/hash/hashing.cu.o] Error 1 [2021-01-31T08:12:12.238Z] make[1]: *** [CMakeFiles/Makefile2:220: CMakeFiles/cudf_hash.dir/all] Error 2 ``` In any case, it is less than ideal for the function to be completely defined in the header, especially given that the likes of `hashing.cu` are exposed to it (by way of `scatter.cuh`). This commit moves the function definition to a separate translation unit, without changing implementation or interface. Authors: - MithunR (@mythrocks) Approvers: - @nvdbaranec - Mike Wilson (@hyperbolic2346) - David (@davidwendt) URL: https://github.com/rapidsai/cudf/pull/7266 --- cpp/include/cudf/lists/detail/scatter.cuh | 14 ++++--- cpp/include/cudf/lists/detail/utilities.cuh | 46 --------------------- cpp/src/rolling/rolling_detail.cuh | 5 ++- 3 files changed, 12 insertions(+), 53 deletions(-) delete mode 100644 cpp/include/cudf/lists/detail/utilities.cuh diff --git a/cpp/include/cudf/lists/detail/scatter.cuh b/cpp/include/cudf/lists/detail/scatter.cuh index 32f6cc6db7a..d737af9c532 100644 --- a/cpp/include/cudf/lists/detail/scatter.cuh +++ b/cpp/include/cudf/lists/detail/scatter.cuh @@ -19,8 +19,8 @@ #include #include #include +#include #include -#include #include #include #include @@ -333,7 +333,8 @@ struct list_child_constructor { auto source_lists = cudf::detail::lists_column_device_view(*source_column_device_view); auto target_lists = cudf::detail::lists_column_device_view(*target_column_device_view); - auto const num_child_rows{get_num_child_rows(list_offsets, stream)}; + auto const num_child_rows{ + cudf::detail::get_value(list_offsets, list_offsets.size() - 1, stream)}; auto const child_null_mask = source_lists_column_view.child().nullable() || target_lists_column_view.child().nullable() @@ -427,7 +428,8 @@ struct list_child_constructor { auto source_lists = cudf::detail::lists_column_device_view(*source_column_device_view); auto target_lists = cudf::detail::lists_column_device_view(*target_column_device_view); - int32_t num_child_rows{get_num_child_rows(list_offsets, stream)}; + auto const num_child_rows{ + cudf::detail::get_value(list_offsets, list_offsets.size() - 1, stream)}; auto string_views = rmm::device_vector(num_child_rows); @@ -516,7 +518,8 @@ struct list_child_constructor { auto source_lists = cudf::detail::lists_column_device_view(*source_column_device_view); auto target_lists = cudf::detail::lists_column_device_view(*target_column_device_view); - auto num_child_rows = get_num_child_rows(list_offsets, stream); + auto const num_child_rows{ + cudf::detail::get_value(list_offsets, list_offsets.size() - 1, stream)}; auto child_list_views = rmm::device_uvector(num_child_rows, stream, mr); @@ -621,7 +624,8 @@ struct list_child_constructor { auto const source_structs = source_lists_column_view.child(); auto const target_structs = target_lists_column_view.child(); - auto const num_child_rows = get_num_child_rows(list_offsets, stream); + auto const num_child_rows{ + cudf::detail::get_value(list_offsets, list_offsets.size() - 1, stream)}; auto const num_struct_members = std::distance(source_structs.child_begin(), source_structs.child_end()); diff --git a/cpp/include/cudf/lists/detail/utilities.cuh b/cpp/include/cudf/lists/detail/utilities.cuh deleted file mode 100644 index ccee9b0d5d9..00000000000 --- a/cpp/include/cudf/lists/detail/utilities.cuh +++ /dev/null @@ -1,46 +0,0 @@ -/* - * Copyright (c) 2021, 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 - -namespace cudf { -namespace detail { - -/** - * @brief Fetch the number of rows in a lists column's child given its offsets column. - * - * @param[in] list_offsets Offsets child of a lists column - * @param[in] stream The cuda-stream to synchronize on, when reading from device memory - * @return cudf::size_type The number of child rows in the lists column - */ -static cudf::size_type get_num_child_rows(cudf::column_view const& list_offsets, - rmm::cuda_stream_view stream) -{ - // Number of rows in child-column == last offset value. - cudf::size_type num_child_rows{}; - CUDA_TRY(cudaMemcpyAsync(&num_child_rows, - list_offsets.data() + list_offsets.size() - 1, - sizeof(cudf::size_type), - cudaMemcpyDeviceToHost, - stream.value())); - stream.synchronize(); - return num_child_rows; -} - -} // namespace detail -} // namespace cudf diff --git a/cpp/src/rolling/rolling_detail.cuh b/cpp/src/rolling/rolling_detail.cuh index 8a0f5f8002d..2ede50b468a 100644 --- a/cpp/src/rolling/rolling_detail.cuh +++ b/cpp/src/rolling/rolling_detail.cuh @@ -29,6 +29,7 @@ #include #include #include +#include #include #include #include @@ -36,7 +37,6 @@ #include #include #include -#include #include #include #include @@ -983,7 +983,8 @@ struct rolling_window_launcher { // This accounts for the `0` added by default to the offsets // column, marking the beginning of the column. - auto num_child_rows = get_num_child_rows(offsets, stream); + auto const num_child_rows{ + cudf::detail::get_value(offsets, offsets.size() - 1, stream)}; auto scatter_values = make_fixed_width_column(size_data_type, offsets.size(), mask_state::UNALLOCATED, stream, mr); From 369ec98d6bca63a1689cc6ec56cf6d1afac9fcdd Mon Sep 17 00:00:00 2001 From: Karthikeyan <6488848+karthikeyann@users.noreply.github.com> Date: Thu, 4 Feb 2021 11:31:56 +0530 Subject: [PATCH 3/8] Add Segmented sort (#7122) addresses part of #6541 Segment sort of lists - [x] lists_column_view segmented_sort - [x] numerical types (cub segmented sort limitation) - [x] sort_lists(table_view) - [x] unit tests closes #4603 Segmented sort - [x] segmented_sort - [x] unit tests. Authors: - Karthikeyan (@karthikeyann) Approvers: - AJ Schmidt (@ajschmidt8) - Keith Kraus (@kkraus14) - Jake Hemstad (@jrhemstad) - Conor Hoekstra (@codereport) URL: https://github.com/rapidsai/cudf/pull/7122 --- conda/recipes/libcudf/meta.yaml | 3 +- cpp/include/cudf/detail/sorting.hpp | 29 ++- cpp/include/cudf/lists/list_device_view.cuh | 25 +- cpp/include/cudf/lists/sorting.hpp | 59 +++++ cpp/include/cudf/sorting.hpp | 61 ++++- cpp/src/lists/count_elements.cu | 9 +- cpp/src/lists/segmented_sort.cu | 270 +++++++++++++++++++ cpp/src/sort/segmented_sort.cu | 141 ++++++++++ cpp/tests/CMakeLists.txt | 4 +- cpp/tests/lists/sort_lists_tests.cpp | 182 +++++++++++++ cpp/tests/sort/segmented_sort_tests.cpp | 272 ++++++++++++++++++++ 11 files changed, 1042 insertions(+), 13 deletions(-) create mode 100644 cpp/include/cudf/lists/sorting.hpp create mode 100644 cpp/src/lists/segmented_sort.cu create mode 100644 cpp/src/sort/segmented_sort.cu create mode 100644 cpp/tests/lists/sort_lists_tests.cpp create mode 100644 cpp/tests/sort/segmented_sort_tests.cpp diff --git a/conda/recipes/libcudf/meta.yaml b/conda/recipes/libcudf/meta.yaml index 1d660e2cd74..a1953a2d358 100644 --- a/conda/recipes/libcudf/meta.yaml +++ b/conda/recipes/libcudf/meta.yaml @@ -1,4 +1,4 @@ -# Copyright (c) 2018-2020, NVIDIA CORPORATION. +# Copyright (c) 2018-2021, NVIDIA CORPORATION. {% set version = environ.get('GIT_DESCRIBE_TAG', '0.0.0.dev').lstrip('v') + environ.get('VERSION_SUFFIX', '') %} {% set minor_version = version.split('.')[0] + '.' + version.split('.')[1] %} @@ -128,6 +128,7 @@ test: - test -f $PREFIX/include/cudf/lists/contains.hpp - test -f $PREFIX/include/cudf/lists/gather.hpp - test -f $PREFIX/include/cudf/lists/lists_column_view.hpp + - test -f $PREFIX/include/cudf/lists/sorting.hpp - test -f $PREFIX/include/cudf/merge.hpp - test -f $PREFIX/include/cudf/null_mask.hpp - test -f $PREFIX/include/cudf/partitioning.hpp diff --git a/cpp/include/cudf/detail/sorting.hpp b/cpp/include/cudf/detail/sorting.hpp index 0ac20ed3c94..3127a5f89f1 100644 --- a/cpp/include/cudf/detail/sorting.hpp +++ b/cpp/include/cudf/detail/sorting.hpp @@ -1,5 +1,5 @@ /* - * Copyright (c) 2019-2020, NVIDIA CORPORATION. + * Copyright (c) 2019-2021, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -63,5 +63,32 @@ std::unique_ptr sort_by_key( rmm::cuda_stream_view stream = rmm::cuda_stream_default, rmm::mr::device_memory_resource* mr = rmm::mr::get_current_device_resource()); +/** + * @copydoc cudf::segmented_sorted_order + * + * @param[in] stream CUDA stream used for device memory operations and kernel launches. + */ +std::unique_ptr segmented_sorted_order( + table_view const& keys, + column_view const& segment_offsets, + std::vector const& column_order = {}, + std::vector const& null_precedence = {}, + rmm::cuda_stream_view stream = rmm::cuda_stream_default, + rmm::mr::device_memory_resource* mr = rmm::mr::get_current_device_resource()); + +/** + * @copydoc cudf::segmented_sort_by_key + * + * @param[in] stream CUDA stream used for device memory operations and kernel launches. + */ +std::unique_ptr
segmented_sort_by_key( + table_view const& values, + table_view const& keys, + column_view const& segment_offsets, + std::vector const& column_order = {}, + std::vector const& null_precedence = {}, + rmm::cuda_stream_view stream = rmm::cuda_stream_default, + rmm::mr::device_memory_resource* mr = rmm::mr::get_current_device_resource()); + } // namespace detail } // namespace cudf diff --git a/cpp/include/cudf/lists/list_device_view.cuh b/cpp/include/cudf/lists/list_device_view.cuh index 824b10ced83..81109bedaed 100644 --- a/cpp/include/cudf/lists/list_device_view.cuh +++ b/cpp/include/cudf/lists/list_device_view.cuh @@ -1,5 +1,5 @@ /* - * Copyright (c) 2020, NVIDIA CORPORATION. + * Copyright (c) 2020-2021, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -190,4 +190,27 @@ class list_device_view { }; }; +/** + * @brief returns size of the list by row index + * + */ +struct list_size_functor { + column_device_view const d_column; + CUDA_HOST_DEVICE_CALLABLE list_size_functor(column_device_view const& d_col) : d_column(d_col) + { +#if defined(__CUDA_ARCH__) + release_assert(d_col.type().id() == type_id::LIST && "Only list type column is supported"); +#else + CUDF_EXPECTS(d_col.type().id() == type_id::LIST, "Only list type column is supported"); +#endif + } + CUDA_DEVICE_CALLABLE size_type operator()(size_type idx) + { + if (d_column.is_null(idx)) return size_type{0}; + auto d_offsets = + d_column.child(lists_column_view::offsets_column_index).data() + d_column.offset(); + return d_offsets[idx + 1] - d_offsets[idx]; + } +}; + } // namespace cudf diff --git a/cpp/include/cudf/lists/sorting.hpp b/cpp/include/cudf/lists/sorting.hpp new file mode 100644 index 00000000000..e27f3d03d86 --- /dev/null +++ b/cpp/include/cudf/lists/sorting.hpp @@ -0,0 +1,59 @@ +/* + * Copyright (c) 2021, 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 + +namespace cudf { +namespace lists { +/** + * @addtogroup lists_sort + * @{ + * @file + */ + +/** + * @brief Segmented sort of the elements within a list in each row of a list column. + * + * `source_column` with depth 1 is only supported. + * + * * @code{.pseudo} + * source_column : [{4, 2, 3, 1}, {1, 2, NULL, 4}, {-10, 10, 0}] + * + * Ascending, Null After : [{1, 2, 3, 4}, {1, 2, 4, NULL}, {-10, 0, 10}] + * Ascending, Null Before : [{1, 2, 3, 4}, {NULL, 1, 2, 4}, {-10, 0, 10}] + * Descending, Null After : [{4, 3, 2, 1}, {NULL, 4, 2, 1}, {10, 0, -10}] + * Descending, Null Before : [{4, 3, 2, 1}, {4, 2, 1, NULL}, {10, 0, -10}] + * @endcode + * + * @param source_column View of the list column of numeric types to sort + * @param column_order The desired sort order + * @param null_precedence The desired order of null compared to other elements in the list + * @param mr Device memory resource to allocate any returned objects + * @return list column with elements in each list sorted. + * + */ +std::unique_ptr sort_lists( + lists_column_view const& source_column, + order column_order, + null_order null_precedence, + rmm::mr::device_memory_resource* mr = rmm::mr::get_current_device_resource()); + +/** @} */ // end of group +} // namespace lists +} // namespace cudf diff --git a/cpp/include/cudf/sorting.hpp b/cpp/include/cudf/sorting.hpp index 1116b49c892..2454cfe7c7b 100644 --- a/cpp/include/cudf/sorting.hpp +++ b/cpp/include/cudf/sorting.hpp @@ -1,5 +1,5 @@ /* - * Copyright (c) 2019-2020, NVIDIA CORPORATION. + * Copyright (c) 2019-2021, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -128,7 +128,7 @@ std::unique_ptr
sort( * @param values The table to reorder * @param keys The table that determines the ordering * @param column_order The desired order for each column in `keys`. Size must be - * equal to `input.num_columns()` or empty. If empty, all columns are sorted in + * equal to `keys.num_columns()` or empty. If empty, all columns are sorted in * ascending order. * @param null_precedence The desired order of a null element compared to other * elements for each column in `keys`. Size must be equal to @@ -184,5 +184,62 @@ std::unique_ptr rank( bool percentage, rmm::mr::device_memory_resource* mr = rmm::mr::get_current_device_resource()); +/** + * @brief Returns sorted order after sorting each segment in the table. + * + * If segment_offsets contains values larger than number of rows, behaviour is undefined. + * @throws cudf::logic_error if `segment_offsets` is not `size_type` column. + * + * @param keys The table that determines the ordering of elements in each segment + * @param segment_offsets The column of `size_type` type containing start offset index for each + * contiguous segment. + * @param column_order The desired order for each column in `keys`. Size must be + * equal to `keys.num_columns()` or empty. If empty, all columns are sorted in + * ascending order. + * @param null_precedence The desired order of a null element compared to other + * elements for each column in `keys`. Size must be equal to + * `keys.num_columns()` or empty. If empty, all columns will be sorted with + * `null_order::BEFORE`. + * @param mr Device memory resource to allocate any returned objects + * @return sorted order of the segment sorted table . + * + */ +std::unique_ptr segmented_sorted_order( + table_view const& keys, + column_view const& segment_offsets, + std::vector const& column_order = {}, + std::vector const& null_precedence = {}, + rmm::mr::device_memory_resource* mr = rmm::mr::get_current_device_resource()); + +/** + * @brief Performs a lexicographic segmented sort of a table + * + * If segment_offsets contains values larger than number of rows, behaviour is undefined. + * @throws cudf::logic_error if `values.num_rows() != keys.num_rows()`. + * @throws cudf::logic_error if `segment_offsets` is not `size_type` column. + * + * @param values The table to reorder + * @param keys The table that determines the ordering of elements in each segment + * @param segment_offsets The column of `size_type` type containing start offset index for each + * contiguous segment. + * @param column_order The desired order for each column in `keys`. Size must be + * equal to `keys.num_columns()` or empty. If empty, all columns are sorted in + * ascending order. + * @param null_precedence The desired order of a null element compared to other + * elements for each column in `keys`. Size must be equal to + * `keys.num_columns()` or empty. If empty, all columns will be sorted with + * `null_order::BEFORE`. + * @param mr Device memory resource to allocate any returned objects + * @return table with elements in each segment sorted. + * + */ +std::unique_ptr
segmented_sort_by_key( + table_view const& values, + table_view const& keys, + column_view const& segment_offsets, + std::vector const& column_order = {}, + std::vector const& null_precedence = {}, + rmm::mr::device_memory_resource* mr = rmm::mr::get_current_device_resource()); + /** @} */ // end of group } // namespace cudf diff --git a/cpp/src/lists/count_elements.cu b/cpp/src/lists/count_elements.cu index 78549152770..ba366b3a020 100644 --- a/cpp/src/lists/count_elements.cu +++ b/cpp/src/lists/count_elements.cu @@ -20,6 +20,7 @@ #include #include #include +#include #include #include @@ -60,13 +61,7 @@ std::unique_ptr count_elements(lists_column_view const& input, thrust::make_counting_iterator(0), thrust::make_counting_iterator(input.size()), output->mutable_view().begin(), - [d_column] __device__(size_type idx) { - if (d_column.is_null(idx)) return size_type{0}; - auto d_offsets = - d_column.child(lists_column_view::offsets_column_index).data() + - d_column.offset(); - return d_offsets[idx + 1] - d_offsets[idx]; - }); + list_size_functor{d_column}); output->set_null_count(input.null_count()); // reset null count return output; diff --git a/cpp/src/lists/segmented_sort.cu b/cpp/src/lists/segmented_sort.cu new file mode 100644 index 00000000000..5681f7767e0 --- /dev/null +++ b/cpp/src/lists/segmented_sort.cu @@ -0,0 +1,270 @@ +/* + * Copyright (c) 2021, 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 +#include +#include + +#include + +#include + +namespace cudf { +namespace lists { +namespace detail { + +struct SegmentedSortColumn { + template + void SortPairsAscending(KeyT const* keys_in, + KeyT* keys_out, + ValueT const* values_in, + ValueT* values_out, + int num_items, + int num_segments, + OffsetIteratorT begin_offsets, + OffsetIteratorT end_offsets, + rmm::cuda_stream_view stream) + { + rmm::device_buffer d_temp_storage; + size_t temp_storage_bytes = 0; + cub::DeviceSegmentedRadixSort::SortPairs(d_temp_storage.data(), + temp_storage_bytes, + keys_in, + keys_out, + values_in, + values_out, + num_items, + num_segments, + begin_offsets, + end_offsets, + 0, + sizeof(KeyT) * 8, + stream.value()); + d_temp_storage = rmm::device_buffer{temp_storage_bytes, stream}; + + cub::DeviceSegmentedRadixSort::SortPairs(d_temp_storage.data(), + temp_storage_bytes, + keys_in, + keys_out, + values_in, + values_out, + num_items, + num_segments, + begin_offsets, + end_offsets, + 0, + sizeof(KeyT) * 8, + stream.value()); + } + + template + void SortPairsDescending(KeyT const* keys_in, + KeyT* keys_out, + ValueT const* values_in, + ValueT* values_out, + int num_items, + int num_segments, + OffsetIteratorT begin_offsets, + OffsetIteratorT end_offsets, + rmm::cuda_stream_view stream) + { + rmm::device_buffer d_temp_storage; + size_t temp_storage_bytes = 0; + cub::DeviceSegmentedRadixSort::SortPairsDescending(d_temp_storage.data(), + temp_storage_bytes, + keys_in, + keys_out, + values_in, + values_out, + num_items, + num_segments, + begin_offsets, + end_offsets, + 0, + sizeof(KeyT) * 8, + stream.value()); + d_temp_storage = rmm::device_buffer{temp_storage_bytes, stream}; + + cub::DeviceSegmentedRadixSort::SortPairsDescending(d_temp_storage.data(), + temp_storage_bytes, + keys_in, + keys_out, + values_in, + values_out, + num_items, + num_segments, + begin_offsets, + end_offsets, + 0, + sizeof(KeyT) * 8, + stream.value()); + } + + template + std::enable_if_t(), std::unique_ptr> operator()( + column_view const& child, + column_view const& segment_offsets, + order column_order, + null_order null_precedence, + rmm::cuda_stream_view stream, + rmm::mr::device_memory_resource* mr) + { + auto child_table = segmented_sort_by_key(table_view{{child}}, + table_view{{child}}, + segment_offsets, + {column_order}, + {null_precedence}, + stream, + mr); + return std::move(child_table->release().front()); + } + + template + std::enable_if_t(), std::unique_ptr> operator()( + column_view const& child, + column_view const& offsets, + order column_order, + null_order null_precedence, + rmm::cuda_stream_view stream, + rmm::mr::device_memory_resource* mr) + { + auto output = + cudf::detail::allocate_like(child, child.size(), mask_allocation_policy::NEVER, stream, mr); + mutable_column_view mutable_output_view = output->mutable_view(); + + auto keys = [&]() { + if (child.nullable()) { + rmm::device_uvector keys(child.size(), stream); + auto const null_replace_T = null_precedence == null_order::AFTER + ? std::numeric_limits::max() + : std::numeric_limits::min(); + auto device_child = column_device_view::create(child, stream); + auto keys_in = + cudf::detail::make_null_replacement_iterator(*device_child, null_replace_T); + thrust::copy_n(rmm::exec_policy(stream), keys_in, child.size(), keys.begin()); + return keys; + } + return rmm::device_uvector{0, stream}; + }(); + + std::unique_ptr sorted_indices = cudf::make_numeric_column( + data_type(type_to_id()), child.size(), mask_state::UNALLOCATED, stream, mr); + mutable_column_view mutable_indices_view = sorted_indices->mutable_view(); + thrust::sequence(rmm::exec_policy(stream), + mutable_indices_view.begin(), + mutable_indices_view.end(), + 0); + + if (column_order == order::ASCENDING) + SortPairsAscending(child.nullable() ? keys.data() : child.begin(), + mutable_output_view.begin(), + mutable_indices_view.begin(), + mutable_indices_view.begin(), + child.size(), + offsets.size() - 1, + offsets.begin(), + offsets.begin() + 1, + stream); + else + SortPairsDescending(child.nullable() ? keys.data() : child.begin(), + mutable_output_view.begin(), + mutable_indices_view.begin(), + mutable_indices_view.begin(), + child.size(), + offsets.size() - 1, + offsets.begin(), + offsets.begin() + 1, + stream); + std::vector> output_cols; + output_cols.push_back(std::move(output)); + // rearrange the null_mask. + cudf::detail::gather_bitmask(cudf::table_view{{child}}, + mutable_indices_view.begin(), + output_cols, + cudf::detail::gather_bitmask_op::DONT_CHECK, + stream, + mr); + return std::move(output_cols.front()); + } +}; + +std::unique_ptr sort_lists(lists_column_view const& input, + order column_order, + null_order null_precedence, + rmm::cuda_stream_view stream, + rmm::mr::device_memory_resource* mr) +{ + if (input.is_empty()) return empty_like(input.parent()); + auto segment_offsets = + cudf::detail::slice(input.offsets(), {input.offset(), input.offsets().size()}, stream)[0]; + // Copy list offsets. + auto output_offset = allocate_like(segment_offsets, mask_allocation_policy::RETAIN, mr); + thrust::transform(rmm::exec_policy(stream), + segment_offsets.begin(), + segment_offsets.end(), + output_offset->mutable_view().begin(), + [first = segment_offsets.begin()] __device__(auto offset_index) { + return offset_index - *first; + }); + // for numeric columns, calls Faster segmented radix sort path + // for non-numeric columns, calls segmented_sort_by_key. + auto output_child = type_dispatcher(input.child().type(), + SegmentedSortColumn{}, + input.get_sliced_child(stream), + output_offset->view(), + column_order, + null_precedence, + stream, + mr); + + auto null_mask = cudf::detail::copy_bitmask(input.parent(), stream, mr); + + // Assemble list column & return + return make_lists_column(input.size(), + std::move(output_offset), + std::move(output_child), + input.null_count(), + std::move(null_mask)); +} +} // namespace detail + +std::unique_ptr sort_lists(lists_column_view const& input, + order column_order, + null_order null_precedence, + rmm::mr::device_memory_resource* mr) +{ + CUDF_FUNC_RANGE(); + return detail::sort_lists(input, column_order, null_precedence, rmm::cuda_stream_default, mr); +} + +} // namespace lists +} // namespace cudf diff --git a/cpp/src/sort/segmented_sort.cu b/cpp/src/sort/segmented_sort.cu new file mode 100644 index 00000000000..f8b0e311c9c --- /dev/null +++ b/cpp/src/sort/segmented_sort.cu @@ -0,0 +1,141 @@ +/* + * Copyright (c) 2021, 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 +#include +#include +#include + +#include +#include +#include +#include + +namespace cudf { +namespace detail { + +// returns segment indices for each element for all segments. +// first segment begin index = 0, last segment end index = num_rows. +rmm::device_uvector get_segment_indices(size_type num_rows, + column_view const& offsets, + rmm::cuda_stream_view stream) +{ + rmm::device_uvector segment_ids(num_rows, stream); + + auto offset_begin = offsets.begin(); // assumes already offset column contains offset. + auto offsets_minus_one = thrust::make_transform_iterator( + offset_begin, [offset_begin] __device__(auto i) { return i - 1; }); + auto counting_iter = thrust::make_counting_iterator(0); + thrust::lower_bound(rmm::exec_policy(stream), + offsets_minus_one, + offsets_minus_one + offsets.size(), + counting_iter, + counting_iter + segment_ids.size(), + segment_ids.begin()); + return std::move(segment_ids); +} + +std::unique_ptr segmented_sorted_order(table_view const& keys, + column_view const& segment_offsets, + std::vector const& column_order, + std::vector const& null_precedence, + rmm::cuda_stream_view stream, + rmm::mr::device_memory_resource* mr) +{ + CUDF_EXPECTS(segment_offsets.type() == data_type(type_to_id()), + "segment offsets should be size_type"); + // Get segment id of each element in all segments. + auto segment_ids = get_segment_indices(keys.num_rows(), segment_offsets, stream); + + // insert segment id before all columns. + std::vector keys_with_segid; + keys_with_segid.reserve(keys.num_columns() + 1); + keys_with_segid.push_back( + column_view(data_type(type_to_id()), segment_ids.size(), segment_ids.data())); + keys_with_segid.insert(keys_with_segid.end(), keys.begin(), keys.end()); + auto segid_keys = table_view(keys_with_segid); + + auto prepend_default = [](auto const& vector, auto default_value) { + if (vector.empty()) return vector; + std::remove_cv_t> pre_vector; + pre_vector.reserve(pre_vector.size() + 1); + pre_vector.push_back(default_value); + pre_vector.insert(pre_vector.end(), vector.begin(), vector.end()); + return pre_vector; + }; + auto child_column_order = prepend_default(column_order, order::ASCENDING); + auto child_null_precedence = prepend_default(null_precedence, null_order::AFTER); + // return sorted order of child columns + return detail::sorted_order(segid_keys, child_column_order, child_null_precedence, stream, mr); +} + +std::unique_ptr
segmented_sort_by_key(table_view const& values, + table_view const& keys, + column_view const& segment_offsets, + std::vector const& column_order, + std::vector const& null_precedence, + rmm::cuda_stream_view stream, + rmm::mr::device_memory_resource* mr) +{ + CUDF_EXPECTS(values.num_rows() == keys.num_rows(), + "Mismatch in number of rows for values and keys"); + auto sorted_order = segmented_sorted_order(keys, + segment_offsets, + column_order, + null_precedence, + stream, + rmm::mr::get_current_device_resource()); + + // Gather segmented sort of child value columns` + return detail::gather(values, + sorted_order->view(), + out_of_bounds_policy::DONT_CHECK, + detail::negative_index_policy::NOT_ALLOWED, + stream, + mr); +} +} // namespace detail + +std::unique_ptr
segmented_sort_by_key(table_view const& values, + table_view const& keys, + column_view const& segment_offsets, + std::vector const& column_order, + std::vector const& null_precedence, + rmm::mr::device_memory_resource* mr) +{ + CUDF_FUNC_RANGE(); + return detail::segmented_sort_by_key( + values, keys, segment_offsets, column_order, null_precedence, rmm::cuda_stream_default, mr); +} + +} // namespace cudf diff --git a/cpp/tests/CMakeLists.txt b/cpp/tests/CMakeLists.txt index 642b81ca657..55b1d50767f 100644 --- a/cpp/tests/CMakeLists.txt +++ b/cpp/tests/CMakeLists.txt @@ -362,6 +362,7 @@ ConfigureTest(JSON_TEST "${JSON_TEST_SRC}") # - sort tests ------------------------------------------------------------------------------------ set(SORT_TEST_SRC + "${CMAKE_CURRENT_SOURCE_DIR}/sort/segmented_sort_tests.cpp" "${CMAKE_CURRENT_SOURCE_DIR}/sort/sort_test.cpp" "${CMAKE_CURRENT_SOURCE_DIR}/sort/rank_test.cpp") @@ -664,7 +665,8 @@ ConfigureTest(AST_TEST "${AST_TEST_SRC}") set(LISTS_TEST_SRC "${CMAKE_CURRENT_SOURCE_DIR}/lists/contains_tests.cpp" "${CMAKE_CURRENT_SOURCE_DIR}/lists/count_elements_tests.cpp" - "${CMAKE_CURRENT_SOURCE_DIR}/lists/extract_tests.cpp") + "${CMAKE_CURRENT_SOURCE_DIR}/lists/extract_tests.cpp" + "${CMAKE_CURRENT_SOURCE_DIR}/lists/sort_lists_tests.cpp") ConfigureTest(LISTS_TEST "${LISTS_TEST_SRC}") diff --git a/cpp/tests/lists/sort_lists_tests.cpp b/cpp/tests/lists/sort_lists_tests.cpp new file mode 100644 index 00000000000..ac73297f088 --- /dev/null +++ b/cpp/tests/lists/sort_lists_tests.cpp @@ -0,0 +1,182 @@ +/* + * Copyright (c) 2021, 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 + +template +using LCW = cudf::test::lists_column_wrapper; +using cudf::lists_column_view; +using cudf::lists::sort_lists; + +namespace cudf { +namespace test { + +template +struct SortLists : public BaseFixture { +}; + +TYPED_TEST_CASE(SortLists, NumericTypes); +using SortListsInt = SortLists; + +/* +empty case + empty list + single row with empty list + multi row with empty lists +single case + single list with single element + single list with multi element +normal case without nulls +Null cases + null rows + null elements in list. +Error: + depth>1 +*/ +TYPED_TEST(SortLists, NoNull) +{ + using T = TypeParam; + + // List + LCW list{{3, 2, 1, 4}, {5}, {10, 8, 9}, {6, 7}}; + + // Ascending + // LCW order{{2, 1, 0, 3}, {0}, {1, 2, 0}, {0, 1}}; + LCW expected{{1, 2, 3, 4}, {5}, {8, 9, 10}, {6, 7}}; + auto results = sort_lists(lists_column_view{list}, order::ASCENDING, null_order::AFTER); + CUDF_TEST_EXPECT_COLUMNS_EQUAL(results->view(), expected); + + results = sort_lists(lists_column_view{list}, order::ASCENDING, null_order::BEFORE); + CUDF_TEST_EXPECT_COLUMNS_EQUAL(results->view(), expected); + + // Descending + // LCW order{{3, 0, 1, 2}, {0}, {0, 1, 2}, {1, 0}}; + LCW expected2{{4, 3, 2, 1}, {5}, {10, 9, 8}, {7, 6}}; + results = sort_lists(lists_column_view{list}, order::DESCENDING, null_order::AFTER); + CUDF_TEST_EXPECT_COLUMNS_EQUAL(results->view(), expected2); + + results = sort_lists(lists_column_view{list}, order::DESCENDING, null_order::BEFORE); + CUDF_TEST_EXPECT_COLUMNS_EQUAL(results->view(), expected2); +} + +TYPED_TEST(SortLists, Null) +{ + using T = TypeParam; + if (std::is_same::value) return; + std::vector valids_o{1, 1, 0, 1}; + std::vector valids_a{1, 1, 1, 0}; + std::vector valids_b{0, 1, 1, 1}; + + // List + LCW list{{{3, 2, 4, 1}, valids_o.begin()}, {5}, {10, 8, 9}, {6, 7}}; + // LCW order{{2, 1, 3, 0}, {0}, {1, 2, 0}, {0, 1}}; + LCW expected1{{{1, 2, 3, 4}, valids_a.begin()}, {5}, {8, 9, 10}, {6, 7}}; + LCW expected2{{{4, 1, 2, 3}, valids_b.begin()}, {5}, {8, 9, 10}, {6, 7}}; + auto results = sort_lists(lists_column_view{list}, order::ASCENDING, null_order::AFTER); + CUDF_TEST_EXPECT_COLUMNS_EQUAL(results->view(), expected1); + + results = sort_lists(lists_column_view{list}, order::ASCENDING, null_order::BEFORE); + CUDF_TEST_EXPECT_COLUMNS_EQUAL(results->view(), expected2); + + // Descending + // LCW order{{3, 0, 1, 2}, {0}, {0, 1, 2}, {1, 0}}; + LCW expected3{{{4, 3, 2, 1}, valids_b.begin()}, {5}, {10, 9, 8}, {7, 6}}; + LCW expected4{{{3, 2, 1, 4}, valids_a.begin()}, {5}, {10, 9, 8}, {7, 6}}; + results = sort_lists(lists_column_view{list}, order::DESCENDING, null_order::AFTER); + CUDF_TEST_EXPECT_COLUMNS_EQUAL(results->view(), expected3); + + results = sort_lists(lists_column_view{list}, order::DESCENDING, null_order::BEFORE); + CUDF_TEST_EXPECT_COLUMNS_EQUAL(results->view(), expected4); +} + +TEST_F(SortListsInt, Empty) +{ + using T = int; + LCW l1{}; + LCW l2{LCW{}}; + LCW l3{LCW{}, LCW{}}; + + auto results = sort_lists(lists_column_view{l1}, {}, {}); + CUDF_TEST_EXPECT_COLUMNS_EQUAL(results->view(), l1); + results = sort_lists(lists_column_view{l2}, {}, {}); + CUDF_TEST_EXPECT_COLUMNS_EQUAL(results->view(), l2); + results = sort_lists(lists_column_view{l3}, {}, {}); + CUDF_TEST_EXPECT_COLUMNS_EQUAL(results->view(), l3); +} + +TEST_F(SortListsInt, Single) +{ + using T = int; + LCW l1{{1}}; + LCW l2{{1, 2, 3}}; + + auto results = sort_lists(lists_column_view{l1}, {}, {}); + CUDF_TEST_EXPECT_COLUMNS_EQUAL(results->view(), l1); + results = sort_lists(lists_column_view{l2}, {}, {}); + CUDF_TEST_EXPECT_COLUMNS_EQUAL(results->view(), l2); +} + +TEST_F(SortListsInt, NullRows) +{ + using T = int; + std::vector valids{0, 1, 0}; + LCW l1{{{1, 2, 3}, {4, 5, 6}, {7}}, valids.begin()}; // offset 0, 0, 3, 3 + + auto results = sort_lists(lists_column_view{l1}, {}, {}); + CUDF_TEST_EXPECT_COLUMNS_EQUAL(results->view(), l1); +} + +/* +// Disabling this test. +// Reason: After this exception "cudaErrorAssert device-side assert triggered", further tests fail +TEST_F(SortListsInt, Depth) +{ + using T = int; + LCW l1{LCW{{1, 2}, {3}}, LCW{{4, 5}}}; + // device exception + EXPECT_THROW(sort_lists(lists_column_view{l1}, {}, {}), std::exception); +} +*/ + +TEST_F(SortListsInt, Sliced) +{ + using T = int; + LCW l1{{1, 2, 3, 4}, {5, 6, 7}, {8, 9}, {10}}; + auto sliced_list = cudf::slice(l1, {1, 4})[0]; + + auto results = sort_lists(lists_column_view{sliced_list}, {}, {}); + CUDF_TEST_EXPECT_COLUMNS_EQUAL(results->view(), sliced_list); +} + +} // namespace test +} // namespace cudf diff --git a/cpp/tests/sort/segmented_sort_tests.cpp b/cpp/tests/sort/segmented_sort_tests.cpp new file mode 100644 index 00000000000..e907212c9e8 --- /dev/null +++ b/cpp/tests/sort/segmented_sort_tests.cpp @@ -0,0 +1,272 @@ +/* + * Copyright (c) 2021, 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 + +template +using column_wrapper = cudf::test::fixed_width_column_wrapper; +using LCWstr = cudf::test::lists_column_wrapper; + +namespace cudf { +namespace test { + +template +struct SegmentedSort : public BaseFixture { +}; + +TYPED_TEST_CASE(SegmentedSort, NumericTypes); +using SegmentedSortInt = SegmentedSort; + +/* Summary of test cases. +empty case + key{}, + value{}, + segment_offset{} +single case + keys{1}, value{1} + segmented_offset{0}, {0, 1} +normal case +{8, 9, 2, 3, 2, 2, 4, 1, 7, 5, 6} +{0, 2, 5, 8 11} + without null + with null +corner case + sliced table, + sliced segment_offsets + non-zero start of segment_offsets without offset + non-zero start of segment_offsets with offset +mismatch sizes + keys, values num_rows + order, null_order + segmented_offsets beyond num_rows +//*/ +TEST_F(SegmentedSortInt, Empty) +{ + using T = int; + column_wrapper col_empty{}; + // clang-format off + column_wrapper col1{{8, 9, 2, 3, 2, 2, 4, 1, 7, 5, 6}}; + column_wrapper segments{{0, 2, 5, 8, 11}}; + // clang-format on + table_view table_empty{{col_empty}}; + table_view table_valid{{col1}}; + + CUDF_EXPECT_NO_THROW(cudf::segmented_sort_by_key(table_valid, table_valid, segments)); + CUDF_EXPECT_NO_THROW(cudf::segmented_sort_by_key(table_valid, table_valid, col_empty)); + CUDF_EXPECT_NO_THROW(cudf::segmented_sort_by_key(table_empty, table_empty, segments)); + CUDF_EXPECT_NO_THROW(cudf::segmented_sort_by_key(table_empty, table_empty, col_empty)); + + CUDF_EXPECT_THROW_MESSAGE(cudf::segmented_sort_by_key(table_empty, table_valid, segments), + "Mismatch in number of rows for values and keys"); + CUDF_EXPECT_THROW_MESSAGE(cudf::segmented_sort_by_key(table_empty, table_valid, col_empty), + "Mismatch in number of rows for values and keys"); + CUDF_EXPECT_THROW_MESSAGE(cudf::segmented_sort_by_key(table_valid, table_empty, segments), + "Mismatch in number of rows for values and keys"); + CUDF_EXPECT_THROW_MESSAGE(cudf::segmented_sort_by_key(table_valid, table_empty, col_empty), + "Mismatch in number of rows for values and keys"); +} + +TEST_F(SegmentedSortInt, Single) +{ + using T = int; + column_wrapper col1{{1}}; + column_wrapper col3{{8, 9, 2}}; + column_wrapper segments1{{0}}; + column_wrapper segments2{{0, 3}}; + table_view table_1elem{{col1}}; + table_view table_1segm{{col3}}; + CUDF_EXPECT_NO_THROW(cudf::segmented_sort_by_key(table_1elem, table_1elem, segments2)); + CUDF_EXPECT_NO_THROW(cudf::segmented_sort_by_key(table_1elem, table_1elem, segments1)); + CUDF_EXPECT_NO_THROW(cudf::segmented_sort_by_key(table_1segm, table_1segm, segments2)); + CUDF_EXPECT_NO_THROW(cudf::segmented_sort_by_key(table_1segm, table_1segm, segments1)); +} + +TYPED_TEST(SegmentedSort, NoNull) +{ + using T = TypeParam; + + // segments {0 1 2} {3 4} {5} {6 7 8 9 10}{11 12}{13}{14 15} + column_wrapper col1{{10, 36, 14, 32, 49, 23, 10, 34, 12, 45, 12, 37, 43, 26, 21, 16}}; + column_wrapper col2{{10, 63, 41, 23, 94, 32, 10, 43, 21, 54, 22, 73, 34, 62, 12, 61}}; + // segment sorted order {0 2 1} {3 4} {5} {6 8 10 7 9}{11 12}{13}{15 16} + column_wrapper segments{0, 3, 5, 5, 5, 6, 11, 13, 14, 16}; + table_view input1{{col1}}; + table_view input2{{col1, col2}}; + + // Ascending + column_wrapper col1_asc{{10, 14, 36, 32, 49, 23, 10, 12, 12, 34, 45, 37, 43, 26, 16, 21}}; + + auto results = cudf::segmented_sort_by_key(input1, input1, segments, {order::ASCENDING}); + CUDF_TEST_EXPECT_TABLES_EQUAL(results->view(), table_view{{col1_asc}}); + + column_wrapper col1_des{{36, 14, 10, 49, 32, 23, 45, 34, 12, 12, 10, 43, 37, 26, 21, 16}}; + results = cudf::segmented_sort_by_key(input1, input1, segments, {order::DESCENDING}); + CUDF_TEST_EXPECT_TABLES_EQUAL(results->view(), table_view{{col1_des}}); + + column_wrapper col1_12_asc{{10, 14, 36, 32, 49, 23, 10, 12, 12, 34, 45, 37, 43, 26, 16, 21}}; + column_wrapper col2_12_asc{{10, 41, 63, 23, 94, 32, 10, 21, 22, 43, 54, 73, 34, 62, 61, 12}}; + column_wrapper col2_12_des{{10, 41, 63, 23, 94, 32, 10, 22, 21, 43, 54, 73, 34, 62, 61, 12}}; + + table_view expected12_aa{{col1_12_asc, col2_12_asc}}; + results = cudf::segmented_sort_by_key(input2, input2, segments, {}); + CUDF_TEST_EXPECT_TABLES_EQUAL(results->view(), expected12_aa); + + table_view expected12_ad{{col1_12_asc, col2_12_des}}; + results = + cudf::segmented_sort_by_key(input2, input2, segments, {order::ASCENDING, order::DESCENDING}); + CUDF_TEST_EXPECT_TABLES_EQUAL(results->view(), expected12_ad); +} + +TYPED_TEST(SegmentedSort, Null) +{ + using T = TypeParam; + if (std::is_same::value) return; + + // segments {0 1 2} {3 4} {5} {6 7 8 9 10}{11 12}{13}{14 15} + column_wrapper col1{{1, 3, 2, 4, 5, 23, 6, 8, 7, 9, 7, 37, 43, 26, 21, 16}, + {1, 1, 0, 1, 1, 1, 1, 1, 1, 0, 1, 1, 1, 1, 1, 1}}; + column_wrapper col2{{0, 0, 0, 1, 1, 4, 5, 5, 21, 5, 22, 6, 6, 7, 8, 8}, + {1, 1, 1, 1, 1, 1, 1, 1, 0, 1, 1, 1, 1, 1, 1, 1}}; + column_wrapper segments{0, 3, 5, 5, 5, 6, 11, 13, 14, 16}; + table_view input1{{col1}}; + table_view input2{{col1, col2}}; + + // Ascending + column_wrapper col1_aa{{1, 3, 2, 4, 5, 23, 6, 7, 7, 8, 9, 37, 43, 26, 16, 21}, + {1, 1, 0, 1, 1, 1, 1, 1, 1, 1, 0, 1, 1, 1, 1, 1}}; + column_wrapper col1_ab{{2, 1, 3, 4, 5, 23, 9, 6, 7, 7, 8, 37, 43, 26, 16, 21}, + {0, 1, 1, 1, 1, 1, 0, 1, 1, 1, 1, 1, 1, 1, 1, 1}}; + + auto results = cudf::segmented_sort_by_key(input1, input1, segments, {}, {null_order::AFTER}); + CUDF_TEST_EXPECT_TABLES_EQUAL(results->view(), table_view{{col1_aa}}); + results = cudf::segmented_sort_by_key(input1, input1, segments, {}, {null_order::BEFORE}); + CUDF_TEST_EXPECT_TABLES_EQUAL(results->view(), table_view{{col1_ab}}); + + // Descending + column_wrapper col1_da{{2, 3, 1, 5, 4, 23, 9, 8, 7, 7, 6, 43, 37, 26, 21, 16}, + {0, 1, 1, 1, 1, 1, 0, 1, 1, 1, 1, 1, 1, 1, 1, 1}}; + column_wrapper col1_db{{3, 1, 2, 5, 4, 23, 8, 7, 7, 6, 9, 43, 37, 26, 21, 16}, + {1, 1, 0, 1, 1, 1, 1, 1, 1, 1, 0, 1, 1, 1, 1, 1}}; + results = + cudf::segmented_sort_by_key(input1, input1, segments, {order::DESCENDING}, {null_order::AFTER}); + CUDF_TEST_EXPECT_TABLES_EQUAL(results->view(), table_view{{col1_da}}); + results = cudf::segmented_sort_by_key( + input1, input1, segments, {order::DESCENDING}, {null_order::BEFORE}); + CUDF_TEST_EXPECT_TABLES_EQUAL(results->view(), table_view{{col1_db}}); + + // second row null order. + column_wrapper col2_12_aa{{0, 0, 0, 1, 1, 4, 5, 22, 21, 5, 5, 6, 6, 7, 8, 8}, + {1, 1, 1, 1, 1, 1, 1, 1, 0, 1, 1, 1, 1, 1, 1, 1}}; + column_wrapper col2_12_ab{{0, 0, 0, 1, 1, 4, 5, 5, 21, 22, 5, 6, 6, 7, 8, 8}, + {1, 1, 1, 1, 1, 1, 1, 1, 0, 1, 1, 1, 1, 1, 1, 1}}; + table_view expected12_aa{{col1_aa, col2_12_aa}}; + table_view expected12_ab{{col1_ab, col2_12_ab}}; + results = cudf::segmented_sort_by_key( + input2, input2, segments, {}, {null_order::AFTER, null_order::AFTER}); + CUDF_TEST_EXPECT_TABLES_EQUAL(results->view(), expected12_aa); + results = cudf::segmented_sort_by_key( + input2, input2, segments, {}, {null_order::BEFORE, null_order::BEFORE}); + CUDF_TEST_EXPECT_TABLES_EQUAL(results->view(), expected12_ab); +} + +TEST_F(SegmentedSortInt, NonZeroSegmentsStart) +{ + using T = int; + // clang-format off + column_wrapper col1{{8, 9, 2, 3, 2, 2, 4, 1, 7, 5, 6}}; + column_wrapper segments1{{0, 2, 5, 8, 11}}; + column_wrapper segments2{{ 2, 5, 8, 11}}; + column_wrapper segments3{{ 6, 8, 11}}; + column_wrapper expected1{{0, 1, 2, 4, 3, 7, 5, 6, 9, 10, 8}}; + column_wrapper expected2{{0, 1, 2, 4, 3, 7, 5, 6, 9, 10, 8}}; + column_wrapper expected3{{2, 4, 5, 3, 0, 1, 7, 6, 9, 10, 8}}; + // clang-format on + table_view input{{col1}}; + auto results = cudf::detail::segmented_sorted_order(input, segments1); + CUDF_TEST_EXPECT_COLUMNS_EQUAL(results->view(), expected1); + results = cudf::detail::segmented_sorted_order(input, segments2); + CUDF_TEST_EXPECT_COLUMNS_EQUAL(results->view(), expected2); + results = cudf::detail::segmented_sorted_order(input, segments3); + CUDF_TEST_EXPECT_COLUMNS_EQUAL(results->view(), expected3); +} + +TEST_F(SegmentedSortInt, Sliced) +{ + using T = int; + // clang-format off + column_wrapper col1{{8, 9, 2, 3, 2, 2, 4, 1, 7, 5, 6}}; + // sliced 2, 2, 4, 1, 7, 5, 6 + column_wrapper segments1{{0, 2, 5}}; + column_wrapper segments2{{-4, 0, 2, 5}}; + column_wrapper segments3{{ 7}}; + column_wrapper expected1{{0, 1, 3, 2, 4, 5, 6}}; + column_wrapper expected2{{0, 1, 3, 2, 4, 5, 6}}; + column_wrapper expected3{{3, 0, 1, 2, 5, 6, 4}}; + // clang-format on + auto slice = cudf::slice(col1, {4, 11})[0]; // 7 elements + table_view input{{slice}}; + auto seg_slice = cudf::slice(segments2, {2, 4})[0]; // 2 elements + + // sliced input + auto results = cudf::detail::segmented_sorted_order(input, segments1); + CUDF_TEST_EXPECT_COLUMNS_EQUAL(results->view(), expected1); + // sliced input and sliced segment + results = cudf::detail::segmented_sorted_order(input, seg_slice); + CUDF_TEST_EXPECT_COLUMNS_EQUAL(results->view(), expected2); + // sliced input, segment end. + results = cudf::detail::segmented_sorted_order(input, segments3); + CUDF_TEST_EXPECT_COLUMNS_EQUAL(results->view(), expected3); +} + +TEST_F(SegmentedSortInt, ErrorsMismatchArgSizes) +{ + using T = int; + column_wrapper col1{{1, 2, 3, 4}}; + column_wrapper col2{{5, 6, 7, 8, 9}}; + table_view input1{{col1}}; + + // Mismatch order sizes + EXPECT_THROW( + cudf::segmented_sort_by_key(input1, input1, col2, {order::ASCENDING, order::ASCENDING}, {}), + logic_error); + // Mismatch null precedence sizes + EXPECT_THROW( + cudf::segmented_sort_by_key(input1, input1, col2, {}, {null_order::AFTER, null_order::AFTER}), + logic_error); + // Both + EXPECT_THROW(cudf::segmented_sort_by_key(input1, + input1, + col2, + {order::ASCENDING, order::ASCENDING}, + {null_order::AFTER, null_order::AFTER}), + logic_error); + // segmented_offsets beyond num_rows - undefined behaviour, no throw. + CUDF_EXPECT_NO_THROW(cudf::segmented_sort_by_key(input1, input1, col2)); +} + +} // namespace test +} // namespace cudf From 4f87a59e4fa9303ce67ade3a7d572ce73dca12a9 Mon Sep 17 00:00:00 2001 From: Vukasin Milovanovic Date: Wed, 3 Feb 2021 23:16:13 -0800 Subject: [PATCH 4/8] Throw if bool column would cause incorrect result when writing to ORC (#7261) Issue #6763 Authors: - Vukasin Milovanovic (@vuule) Approvers: - Ram (Ramakrishna Prabhu) (@rgsl888prabhu) - @nvdbaranec - GALI PREM SAGAR (@galipremsagar) - Keith Kraus (@kkraus14) URL: https://github.com/rapidsai/cudf/pull/7261 --- cpp/src/io/orc/writer_impl.cu | 30 ++++++++++++++++++++++++++++++ python/cudf/cudf/tests/test_orc.py | 29 +++++++++++++++++++++++++++++ 2 files changed, 59 insertions(+) diff --git a/cpp/src/io/orc/writer_impl.cu b/cpp/src/io/orc/writer_impl.cu index f1f3d1de9ed..7198539a460 100644 --- a/cpp/src/io/orc/writer_impl.cu +++ b/cpp/src/io/orc/writer_impl.cu @@ -516,6 +516,11 @@ std::vector writer::impl::gather_streams(orc_column_view *columns, return streams; } +struct segmented_valid_cnt_input { + bitmask_type const *mask; + std::vector indices; +}; + rmm::device_buffer writer::impl::encode_columns(orc_column_view *columns, size_t num_columns, size_t num_rows, @@ -555,6 +560,7 @@ rmm::device_buffer writer::impl::encode_columns(orc_column_view *columns, // Initialize column chunks' descriptions size_t stripe_start = 0; size_t stripe_id = 0; + std::map validity_check_inputs; for (size_t j = 0; j < num_rowgroups; j++) { for (size_t i = 0; i < num_columns; i++) { auto *ck = &chunks[j * num_columns + i]; @@ -578,6 +584,20 @@ rmm::device_buffer writer::impl::encode_columns(orc_column_view *columns, } ck->scale = columns[i].clockscale(); + // Only need to check row groups that end within the stripe + if (ck->type_kind == TypeKind::BOOLEAN && columns[i].nullable() && + j + 1 != stripe_start + stripe_list[stripe_id]) { + auto curr_cnt_in = validity_check_inputs.find(i); + if (curr_cnt_in == validity_check_inputs.end()) { + bool unused; + // add new object + std::tie(curr_cnt_in, unused) = validity_check_inputs.insert({i, {columns[i].nulls()}}); + } + // append row group start and end to existing object + curr_cnt_in->second.indices.push_back(ck->start_row); + curr_cnt_in->second.indices.push_back(ck->start_row + ck->num_rows); + } + for (int k = 0; k < gpu::CI_NUM_STREAMS; k++) { const auto strm_id = strm_ids[i * gpu::CI_NUM_STREAMS + k]; @@ -631,6 +651,16 @@ rmm::device_buffer writer::impl::encode_columns(orc_column_view *columns, } } + for (auto &cnt_in : validity_check_inputs) { + auto const valid_counts = segmented_count_set_bits(cnt_in.second.mask, cnt_in.second.indices); + CUDF_EXPECTS( + std::none_of(valid_counts.cbegin(), + valid_counts.cend(), + [](auto valid_count) { return valid_count % 8; }), + "There's currently a bug in encoding boolean columns. Suggested workaround is to convert to " + "int8 type. Please see https://github.com/rapidsai/cudf/issues/6763 for more information."); + } + chunks.host_to_device(stream); if (!str_col_ids.empty()) { auto d_stripe_dict = columns[str_col_ids[0]].device_stripe_dict(); diff --git a/python/cudf/cudf/tests/test_orc.py b/python/cudf/cudf/tests/test_orc.py index 85e61acd8e6..cf3da26258e 100644 --- a/python/cudf/cudf/tests/test_orc.py +++ b/python/cudf/cudf/tests/test_orc.py @@ -571,6 +571,9 @@ def normalized_equals(value1, value2): @pytest.mark.parametrize("nrows", [1, 100, 6000000]) def test_orc_write_statistics(tmpdir, datadir, nrows): supported_stat_types = supported_numpy_dtypes + ["str"] + # Can't write random bool columns until issue #6763 is fixed + if nrows == 6000000: + supported_stat_types.remove("bool") # Make a dataframe gdf = cudf.DataFrame( @@ -670,3 +673,29 @@ def test_orc_reader_gmt_timestamps(datadir): pdf = orcfile.read().to_pandas() gdf = cudf.read_orc(path, engine="cudf").to_pandas() assert_eq(pdf, gdf) + + +def test_orc_bool_encode_fail(): + np.random.seed(0) + + # Generate a boolean column longer than a single stripe + fail_df = cudf.DataFrame({"col": gen_rand_series("bool", 600000)}) + # Invalidate the first row in the second stripe to break encoding + fail_df["col"][500000] = None + + # Should throw instead of generating a file that is incompatible + # with other readers (see issue #6763) + with pytest.raises(RuntimeError): + fail_df.to_orc("should_throw.orc") + + # Generate a boolean column that fits into a single stripe + okay_df = cudf.DataFrame({"col": gen_rand_series("bool", 500000)}) + okay_df["col"][500000 - 1] = None + fname = "single_stripe.orc" + # Invalid row is in the last row group of the stripe; + # encoding is assumed to be correct + okay_df.to_orc(fname) + + # Also validate data + pdf = pa.orc.ORCFile(fname).read().to_pandas() + assert_eq(okay_df, pdf) From 110ef3ef012715c20c4334786a94e1b0f9f6b8c5 Mon Sep 17 00:00:00 2001 From: Jason Lowe Date: Thu, 4 Feb 2021 08:19:48 -0600 Subject: [PATCH 5/8] Update JNI for contiguous_split packed results (#7127) This PR requires the libcudf changes in #7096, fixing the Java bindings to `contiguous_split` that are broken by that change. This also adds the ability to create a `ContiguousTable` instance without manifesting a `Table` instance and all `ColumnVector` instances underneath it which should prove useful during Spark's shuffle. Authors: - Jason Lowe (@jlowe) Approvers: - Robert (Bobby) Evans (@revans2) - Alessandro Bellina (@abellina) URL: https://github.com/rapidsai/cudf/pull/7127 --- .../java/ai/rapids/cudf/ContiguousTable.java | 110 +++++++++++----- java/src/main/java/ai/rapids/cudf/Table.java | 54 +++++++- java/src/main/native/CMakeLists.txt | 1 + .../main/native/src/ContiguousTableJni.cpp | 122 ++++++++++++++++++ java/src/main/native/src/CudfJni.cpp | 72 +---------- java/src/main/native/src/TableJni.cpp | 37 +++++- java/src/main/native/src/cudf_jni_apis.hpp | 17 ++- .../java/ai/rapids/cudf/ColumnVectorTest.java | 6 + .../test/java/ai/rapids/cudf/TableTest.java | 28 ++++ 9 files changed, 337 insertions(+), 110 deletions(-) create mode 100644 java/src/main/native/src/ContiguousTableJni.cpp diff --git a/java/src/main/java/ai/rapids/cudf/ContiguousTable.java b/java/src/main/java/ai/rapids/cudf/ContiguousTable.java index 94e44fa9d79..87a3f5f0ddf 100644 --- a/java/src/main/java/ai/rapids/cudf/ContiguousTable.java +++ b/java/src/main/java/ai/rapids/cudf/ContiguousTable.java @@ -1,6 +1,6 @@ /* * - * Copyright (c) 2019-2020, NVIDIA CORPORATION. + * Copyright (c) 2019-2021, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -18,63 +18,96 @@ package ai.rapids.cudf; -import java.util.Arrays; +import java.nio.ByteBuffer; /** * A table that is backed by a single contiguous device buffer. This makes transfers of the data * much simpler. */ public final class ContiguousTable implements AutoCloseable { - private Table table; + private long metadataHandle = 0; + private Table table = null; private DeviceMemoryBuffer buffer; + private ByteBuffer metadataBuffer = null; + private final long rowCount; - //Will be called from JNI - static ContiguousTable fromContiguousColumnViews(long[] columnViewAddresses, - long address, long lengthInBytes, long rmmBufferAddress) { - Table table = null; - ColumnVector[] vectors = new ColumnVector[columnViewAddresses.length]; - DeviceMemoryBuffer buffer = DeviceMemoryBuffer.fromRmm(address, lengthInBytes, rmmBufferAddress); - try { - for (int i = 0; i < vectors.length; i++) { - vectors[i] = ColumnVector.fromViewWithContiguousAllocation(columnViewAddresses[i], buffer); - } - table = new Table(vectors); - ContiguousTable ret = new ContiguousTable(table, buffer); - buffer = null; - table = null; - return ret; - } finally { - if (buffer != null) { - buffer.close(); - } - - for (int i = 0; i < vectors.length; i++) { - if (vectors[i] != null) { - vectors[i].close(); - } - } - - if (table != null) { - table.close(); - } - } + // This method is invoked by JNI + static ContiguousTable fromPackedTable(long metadataHandle, + long dataAddress, + long dataLength, + long rmmBufferAddress, + long rowCount) { + DeviceMemoryBuffer buffer = DeviceMemoryBuffer.fromRmm(dataAddress, dataLength, rmmBufferAddress); + return new ContiguousTable(metadataHandle, buffer, rowCount); } + /** Construct a contiguous table instance given a table and the device buffer backing it. */ ContiguousTable(Table table, DeviceMemoryBuffer buffer) { + this.metadataHandle = createPackedMetadata(table.getNativeView(), + buffer.getAddress(), buffer.getLength()); this.table = table; this.buffer = buffer; + this.rowCount = table.getRowCount(); } - public Table getTable() { + /** + * Construct a contiguous table + * @param metadataHandle address of the cudf packed_table host-based metadata instance + * @param buffer buffer containing the packed table data + * @param rowCount number of rows in the table + */ + ContiguousTable(long metadataHandle, DeviceMemoryBuffer buffer, long rowCount) { + this.metadataHandle = metadataHandle; + this.buffer = buffer; + this.rowCount = rowCount; + } + + /** + * Returns the number of rows in the table. This accessor avoids manifesting + * the Table instance if only the row count is needed. + */ + public long getRowCount() { + return rowCount; + } + + /** Get the table instance, reconstructing it from the metadata if necessary. */ + public synchronized Table getTable() { + if (table == null) { + table = Table.fromPackedTable(getMetadataDirectBuffer(), buffer); + } return table; } + /** Get the device buffer backing the contiguous table data. */ public DeviceMemoryBuffer getBuffer() { return buffer; } + /** + * Get the byte buffer containing the host metadata describing the schema and layout of the + * contiguous table. + *

+ * NOTE: This is a direct byte buffer that is backed by the underlying native metadata instance + * and therefore is only valid to be used while this contiguous table instance is valid. + * Attempts to cache and access the resulting buffer after this instance has been destroyed + * will result in undefined behavior including the possibility of segmentation faults + * or data corruption. + */ + public ByteBuffer getMetadataDirectBuffer() { + if (metadataBuffer == null) { + metadataBuffer = createMetadataDirectBuffer(metadataHandle); + } + return metadataBuffer.asReadOnlyBuffer(); + } + + /** Close the contiguous table instance and its underlying resources. */ @Override public void close() { + if (metadataHandle != 0) { + closeMetadata(metadataHandle); + metadataHandle = 0; + } + if (table != null) { table.close(); table = null; @@ -85,4 +118,13 @@ public void close() { buffer = null; } } + + // create packed metadata for a table backed by a single data buffer + private static native long createPackedMetadata(long tableView, long dataAddress, long dataSize); + + // create a DirectByteBuffer for the packed table metadata + private static native ByteBuffer createMetadataDirectBuffer(long metadataHandle); + + // release the native metadata resources for a packed table + private static native void closeMetadata(long metadataHandle); } diff --git a/java/src/main/java/ai/rapids/cudf/Table.java b/java/src/main/java/ai/rapids/cudf/Table.java index da4c446d9f7..0637ae6de1e 100644 --- a/java/src/main/java/ai/rapids/cudf/Table.java +++ b/java/src/main/java/ai/rapids/cudf/Table.java @@ -27,6 +27,7 @@ import java.io.File; import java.math.BigDecimal; import java.math.RoundingMode; +import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -117,6 +118,11 @@ ColumnVector[] getColumns() { return columns; } + /** Return the native table view handle for this table */ + long getNativeView() { + return nativeHandle; + } + /** * Return the {@link ColumnVector} at the specified index. If you want to keep a reference to * the column around past the life time of the table, you will need to increment the reference @@ -503,7 +509,9 @@ private static native long[] repeatColumnCount(long tableHandle, private static native long[] explode(long tableHandle, int index); - private native long createCudfTableView(long[] nativeColumnViewHandles); + private static native long createCudfTableView(long[] nativeColumnViewHandles); + + private static native long[] columnViewsFromPacked(ByteBuffer metadata, long dataAddress); ///////////////////////////////////////////////////////////////////////////// // TABLE CREATION APIs @@ -1796,6 +1804,50 @@ public static Table convertFromRows(ColumnVector vec, DType ... schema) { return new Table(convertFromRows(vec.getNativeView(), types, scale)); } + /** + * Construct a table from a packed representation. + * @param metadata host-based metadata for the table + * @param data GPU data buffer for the table + * @return table which is zero-copy reconstructed from the packed-form + */ + public static Table fromPackedTable(ByteBuffer metadata, DeviceMemoryBuffer data) { + // Ensure the metadata buffer is direct so it can be passed to JNI + ByteBuffer directBuffer = metadata; + if (!directBuffer.isDirect()) { + directBuffer = ByteBuffer.allocateDirect(metadata.remaining()); + directBuffer.put(metadata); + directBuffer.flip(); + } + + long[] columnViewAddresses = columnViewsFromPacked(directBuffer, data.getAddress()); + ColumnVector[] columns = new ColumnVector[columnViewAddresses.length]; + Table result = null; + try { + for (int i = 0; i < columns.length; i++) { + columns[i] = ColumnVector.fromViewWithContiguousAllocation(columnViewAddresses[i], data); + columnViewAddresses[i] = 0; + } + result = new Table(columns); + } catch (Throwable t) { + for (int i = 0; i < columns.length; i++) { + if (columns[i] != null) { + columns[i].close(); + } + if (columnViewAddresses[i] != 0) { + ColumnView.deleteColumnView(columnViewAddresses[i]); + } + } + throw t; + } + + // close columns to leave the resulting table responsible for freeing underlying columns + for (ColumnVector column : columns) { + column.close(); + } + + return result; + } + ///////////////////////////////////////////////////////////////////////////// // HELPER CLASSES ///////////////////////////////////////////////////////////////////////////// diff --git a/java/src/main/native/CMakeLists.txt b/java/src/main/native/CMakeLists.txt index 6d658b6d80b..614ff155c44 100755 --- a/java/src/main/native/CMakeLists.txt +++ b/java/src/main/native/CMakeLists.txt @@ -308,6 +308,7 @@ set(SOURCE_FILES "src/CudaJni.cpp" "src/ColumnVectorJni.cpp" "src/ColumnViewJni.cpp" + "src/ContiguousTableJni.cpp" "src/HostMemoryBufferNativeUtilsJni.cpp" "src/NvcompJni.cpp" "src/NvtxRangeJni.cpp" diff --git a/java/src/main/native/src/ContiguousTableJni.cpp b/java/src/main/native/src/ContiguousTableJni.cpp new file mode 100644 index 00000000000..352256af450 --- /dev/null +++ b/java/src/main/native/src/ContiguousTableJni.cpp @@ -0,0 +1,122 @@ +/* + * Copyright (c) 2021, 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" + +namespace { + +#define CONTIGUOUS_TABLE_CLASS "ai/rapids/cudf/ContiguousTable" +#define CONTIGUOUS_TABLE_FACTORY_SIG(param_sig) "(" param_sig ")L" CONTIGUOUS_TABLE_CLASS ";" + +jclass Contiguous_table_jclass; +jmethodID From_packed_table_method; + +} // anonymous namespace + +namespace cudf { +namespace jni { + +bool cache_contiguous_table_jni(JNIEnv *env) { + jclass cls = env->FindClass(CONTIGUOUS_TABLE_CLASS); + if (cls == nullptr) { + return false; + } + + From_packed_table_method = + env->GetStaticMethodID(cls, "fromPackedTable", CONTIGUOUS_TABLE_FACTORY_SIG("JJJJJ")); + if (From_packed_table_method == nullptr) { + return false; + } + + // Convert local reference to global so it cannot be garbage collected. + Contiguous_table_jclass = static_cast(env->NewGlobalRef(cls)); + if (Contiguous_table_jclass == nullptr) { + return false; + } + return true; +} + +void release_contiguous_table_jni(JNIEnv *env) { + if (Contiguous_table_jclass != nullptr) { + env->DeleteGlobalRef(Contiguous_table_jclass); + Contiguous_table_jclass = nullptr; + } +} + +jobject contiguous_table_from(JNIEnv *env, cudf::packed_columns &split, long row_count) { + jlong metadata_address = reinterpret_cast(split.metadata_.get()); + jlong data_address = reinterpret_cast(split.gpu_data->data()); + jlong data_size = static_cast(split.gpu_data->size()); + jlong rmm_buffer_address = reinterpret_cast(split.gpu_data.get()); + + jobject contig_table_obj = env->CallStaticObjectMethod( + Contiguous_table_jclass, From_packed_table_method, metadata_address, data_address, data_size, + rmm_buffer_address, row_count); + + if (contig_table_obj != nullptr) { + split.metadata_.release(); + split.gpu_data.release(); + } + + return contig_table_obj; +} + +native_jobjectArray contiguous_table_array(JNIEnv *env, jsize length) { + return native_jobjectArray( + env, env->NewObjectArray(length, Contiguous_table_jclass, nullptr)); +} + +} // namespace jni +} // namespace cudf + +extern "C" { + +JNIEXPORT jlong JNICALL Java_ai_rapids_cudf_ContiguousTable_createPackedMetadata( + JNIEnv *env, jclass, jlong j_table, jlong j_buffer_addr, jlong j_buffer_length) { + JNI_NULL_CHECK(env, j_table, "input table is null", 0); + try { + cudf::jni::auto_set_device(env); + auto table = reinterpret_cast(j_table); + auto data_addr = reinterpret_cast(j_buffer_addr); + auto data_size = static_cast(j_buffer_length); + auto metadata_ptr = + new cudf::packed_columns::metadata(cudf::pack_metadata(*table, data_addr, data_size)); + return reinterpret_cast(metadata_ptr); + } + CATCH_STD(env, 0); +} + +JNIEXPORT jobject JNICALL Java_ai_rapids_cudf_ContiguousTable_createMetadataDirectBuffer( + JNIEnv *env, jclass, jlong j_metadata_ptr) { + JNI_NULL_CHECK(env, j_metadata_ptr, "metadata is null", nullptr); + try { + auto metadata = reinterpret_cast(j_metadata_ptr); + return env->NewDirectByteBuffer(const_cast(metadata->data()), metadata->size()); + } + CATCH_STD(env, nullptr); +} + +JNIEXPORT void JNICALL Java_ai_rapids_cudf_ContiguousTable_closeMetadata(JNIEnv *env, jclass, + jlong j_metadata_ptr) { + JNI_NULL_CHECK(env, j_metadata_ptr, "metadata is null", ); + try { + auto metadata = reinterpret_cast(j_metadata_ptr); + delete metadata; + } + CATCH_STD(env, ); +} + +} // extern "C" diff --git a/java/src/main/native/src/CudfJni.cpp b/java/src/main/native/src/CudfJni.cpp index 0c560833bb1..928e167c4da 100644 --- a/java/src/main/native/src/CudfJni.cpp +++ b/java/src/main/native/src/CudfJni.cpp @@ -1,5 +1,5 @@ /* - * Copyright (c) 2019-2020, NVIDIA CORPORATION. + * Copyright (c) 2019-2021, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -19,7 +19,7 @@ #include #include -#include "jni_utils.hpp" +#include "cudf_jni_apis.hpp" namespace { @@ -45,74 +45,6 @@ constexpr bool is_ptds_enabled{true}; constexpr bool is_ptds_enabled{false}; #endif -static jclass Contiguous_table_jclass; -static jmethodID From_contiguous_column_views; - -#define CONTIGUOUS_TABLE_CLASS "ai/rapids/cudf/ContiguousTable" -#define CONTIGUOUS_TABLE_FACTORY_SIG(param_sig) "(" param_sig ")L" CONTIGUOUS_TABLE_CLASS ";" - -static bool cache_contiguous_table_jni(JNIEnv *env) { - jclass cls = env->FindClass(CONTIGUOUS_TABLE_CLASS); - if (cls == nullptr) { - return false; - } - - From_contiguous_column_views = env->GetStaticMethodID(cls, "fromContiguousColumnViews", - CONTIGUOUS_TABLE_FACTORY_SIG("[JJJJ")); - if (From_contiguous_column_views == nullptr) { - return false; - } - - // Convert local reference to global so it cannot be garbage collected. - Contiguous_table_jclass = static_cast(env->NewGlobalRef(cls)); - if (Contiguous_table_jclass == nullptr) { - return false; - } - return true; -} - -static void release_contiguous_table_jni(JNIEnv *env) { - if (Contiguous_table_jclass != nullptr) { - env->DeleteGlobalRef(Contiguous_table_jclass); - Contiguous_table_jclass = nullptr; - } -} - -jobject contiguous_table_from(JNIEnv *env, cudf::contiguous_split_result &split) { - jlong address = reinterpret_cast(split.all_data->data()); - jlong size = static_cast(split.all_data->size()); - jlong buff_address = reinterpret_cast(split.all_data.get()); - int num_columns = split.table.num_columns(); - cudf::jni::native_jlongArray views(env, num_columns); - for (int i = 0; i < num_columns; i++) { - // TODO Exception handling is not ideal, if no exceptions are thrown ownership of the new cv - // is passed to java. If an exception is thrown we need to free it, but this needs to be - // coordinated with the java side because one column may have changed ownership while - // another may not have. We don't want to double free the view so for now we just let it - // leak because it should be a small amount of host memory. - // - // In the ideal case we would keep the view where it is at, and pass in a pointer to it - // That pointer would then be copied when java takes ownership of it, but that adds an - // extra JNI call that I would like to avoid for performance reasons. - cudf::column_view *cv = new cudf::column_view(split.table.column(i)); - views[i] = reinterpret_cast(cv); - } - - views.commit(); - jobject ret = env->CallStaticObjectMethod(Contiguous_table_jclass, From_contiguous_column_views, - views.get_jArray(), address, size, buff_address); - - if (ret != nullptr) { - split.all_data.release(); - } - return ret; -} - -native_jobjectArray contiguous_table_array(JNIEnv *env, jsize length) { - return native_jobjectArray( - env, env->NewObjectArray(length, Contiguous_table_jclass, nullptr)); -} - static jclass Host_memory_buffer_jclass; static jmethodID Host_buffer_allocate; static jfieldID Host_buffer_address; diff --git a/java/src/main/native/src/TableJni.cpp b/java/src/main/native/src/TableJni.cpp index 20afe12baf9..30222452804 100644 --- a/java/src/main/native/src/TableJni.cpp +++ b/java/src/main/native/src/TableJni.cpp @@ -606,6 +606,38 @@ JNIEXPORT void JNICALL Java_ai_rapids_cudf_Table_deleteCudfTable(JNIEnv *env, jc CATCH_STD(env, ); } +JNIEXPORT jlongArray JNICALL Java_ai_rapids_cudf_Table_columnViewsFromPacked(JNIEnv *env, jclass, + jobject buffer_obj, + jlong j_data_address) { + // The GPU data address can be null when the table is empty, so it is not null-checked here. + JNI_NULL_CHECK(env, buffer_obj, "metadata is null", nullptr); + try { + cudf::jni::auto_set_device(env); + void const *metadata_address = env->GetDirectBufferAddress(buffer_obj); + JNI_NULL_CHECK(env, metadata_address, "metadata buffer address is null", nullptr); + cudf::table_view table = cudf::unpack(static_cast(metadata_address), + reinterpret_cast(j_data_address)); + cudf::jni::native_jlongArray views(env, table.num_columns()); + for (int i = 0; i < table.num_columns(); i++) { + // TODO Exception handling is not ideal, if no exceptions are thrown ownership of the new cv + // is passed to Java. If an exception is thrown we need to free it, but this needs to be + // coordinated with the Java side because one column may have changed ownership while + // another may not have. We don't want to double free the view so for now we just let it + // leak because it should be a small amount of host memory. + // + // In the ideal case we would keep the view where it is at, and pass in a pointer to it + // That pointer would then be copied when Java takes ownership of it, but that adds an + // extra JNI call that I would like to avoid for performance reasons. + cudf::column_view *cv = new cudf::column_view(table.column(i)); + views[i] = reinterpret_cast(cv); + } + views.commit(); + + return views.get_jArray(); + } + CATCH_STD(env, nullptr); +} + JNIEXPORT jlongArray JNICALL Java_ai_rapids_cudf_Table_orderBy(JNIEnv *env, jclass j_class_object, jlong j_input_table, jlongArray j_sort_keys_columns, @@ -1817,11 +1849,12 @@ JNIEXPORT jobjectArray JNICALL Java_ai_rapids_cudf_Table_contiguousSplit(JNIEnv std::vector indices(n_split_indices.data(), n_split_indices.data() + n_split_indices.size()); - std::vector result = cudf::contiguous_split(*n_table, indices); + std::vector result = cudf::contiguous_split(*n_table, indices); cudf::jni::native_jobjectArray n_result = cudf::jni::contiguous_table_array(env, result.size()); for (int i = 0; i < result.size(); i++) { - n_result.set(i, cudf::jni::contiguous_table_from(env, result[i])); + n_result.set(i, cudf::jni::contiguous_table_from(env, result[i].data, + result[i].table.num_rows())); } return n_result.wrapped(); } diff --git a/java/src/main/native/src/cudf_jni_apis.hpp b/java/src/main/native/src/cudf_jni_apis.hpp index a7d955b2bbf..76c7e91d335 100644 --- a/java/src/main/native/src/cudf_jni_apis.hpp +++ b/java/src/main/native/src/cudf_jni_apis.hpp @@ -1,5 +1,5 @@ /* - * Copyright (c) 2019-2020, NVIDIA CORPORATION. + * Copyright (c) 2019-2021, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -23,12 +23,23 @@ namespace cudf { namespace jni { +jlongArray convert_table_for_return(JNIEnv *env, std::unique_ptr &table_result); + +// +// ContiguousTable APIs +// + +bool cache_contiguous_table_jni(JNIEnv *env); -jobject contiguous_table_from(JNIEnv *env, cudf::contiguous_split_result &split); +void release_contiguous_table_jni(JNIEnv *env); + +jobject contiguous_table_from(JNIEnv *env, cudf::packed_columns &split, long row_count); native_jobjectArray contiguous_table_array(JNIEnv *env, jsize length); -jlongArray convert_table_for_return(JNIEnv *env, std::unique_ptr &table_result); +// +// HostMemoryBuffer APIs +// /** * Allocate a HostMemoryBuffer diff --git a/java/src/test/java/ai/rapids/cudf/ColumnVectorTest.java b/java/src/test/java/ai/rapids/cudf/ColumnVectorTest.java index 7806bd1797b..cb1f792b99e 100644 --- a/java/src/test/java/ai/rapids/cudf/ColumnVectorTest.java +++ b/java/src/test/java/ai/rapids/cudf/ColumnVectorTest.java @@ -3737,6 +3737,12 @@ void testConcatListsOfLists() { void testContiguousSplitConstructor() { try (Table tmp = new Table.TestBuilder().column(1, 2).column(3, 4).build(); ContiguousTable ct = tmp.contiguousSplit()[0]) { + // table should not be referencing the device buffer yet + assertEquals(1, ct.getBuffer().getRefCount()); + + // get the table to force it to be instantiated + Table ignored = ct.getTable(); + // one reference for the device buffer itself, two more for the column using it assertEquals(3, ct.getBuffer().getRefCount()); } diff --git a/java/src/test/java/ai/rapids/cudf/TableTest.java b/java/src/test/java/ai/rapids/cudf/TableTest.java index 35be427d0c8..bb9e5e40cb9 100644 --- a/java/src/test/java/ai/rapids/cudf/TableTest.java +++ b/java/src/test/java/ai/rapids/cudf/TableTest.java @@ -35,6 +35,7 @@ import java.io.IOException; import java.math.BigDecimal; import java.math.RoundingMode; +import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; import java.nio.file.Files; import java.util.ArrayList; @@ -1672,9 +1673,13 @@ void testContiguousSplit() { .build()) { splits = t1.contiguousSplit(2, 5, 9); assertEquals(4, splits.length); + assertEquals(2, splits[0].getRowCount()); assertEquals(2, splits[0].getTable().getRowCount()); + assertEquals(3, splits[1].getRowCount()); assertEquals(3, splits[1].getTable().getRowCount()); + assertEquals(4, splits[2].getRowCount()); assertEquals(4, splits[2].getTable().getRowCount()); + assertEquals(1, splits[3].getRowCount()); assertEquals(1, splits[3].getTable().getRowCount()); } finally { if (splits != null) { @@ -1697,9 +1702,13 @@ void testContiguousSplitWithStrings() { .build()) { splits = t1.contiguousSplit(2, 5, 9); assertEquals(4, splits.length); + assertEquals(2, splits[0].getRowCount()); assertEquals(2, splits[0].getTable().getRowCount()); + assertEquals(3, splits[1].getRowCount()); assertEquals(3, splits[1].getTable().getRowCount()); + assertEquals(4, splits[2].getRowCount()); assertEquals(4, splits[2].getTable().getRowCount()); + assertEquals(1, splits[3].getRowCount()); assertEquals(1, splits[3].getTable().getRowCount()); } finally { if (splits != null) { @@ -2164,6 +2173,25 @@ void testSerializationRoundTripSliced() throws IOException { } } + @Test + void testSerializationReconstructFromMetadata() throws IOException { + try (Table t = buildTestTable()) { + ByteArrayOutputStream bout = new ByteArrayOutputStream(); + JCudfSerialization.writeToStream(t, bout, 0, t.getRowCount()); + ByteArrayInputStream bin = new ByteArrayInputStream(bout.toByteArray()); + try (JCudfSerialization.TableAndRowCountPair trcp = JCudfSerialization.readTableFrom(bin)) { + ContiguousTable contigTable = trcp.getContiguousTable(); + DeviceMemoryBuffer oldbuf = contigTable.getBuffer(); + try (DeviceMemoryBuffer newbuf = oldbuf.sliceWithCopy(0, oldbuf.getLength())) { + ByteBuffer metadata = contigTable.getMetadataDirectBuffer(); + try (Table newTable = Table.fromPackedTable(metadata, newbuf)) { + assertTablesAreEqual(t, newTable); + } + } + } + } + } + @Test void testValidityFill() { byte[] buff = new byte[2]; From 1062fbc8a6e53840cf5f671524964904cda6209c Mon Sep 17 00:00:00 2001 From: Rong Ou Date: Thu, 4 Feb 2021 06:34:41 -0800 Subject: [PATCH 6/8] fix java cuFile tests (#7296) Turns out we need version > 5.4 of the junit jupiter engine to support `@TempDir`. - Changed the file mode to match Spark's disk manager. - Changed to use `fstat` to get the file length when appending. - Add tests for when a file already exists. Authors: - Rong Ou (@rongou) Approvers: - Jason Lowe (@jlowe) - Robert (Bobby) Evans (@revans2) URL: https://github.com/rapidsai/cudf/pull/7296 --- java/pom.xml | 2 +- java/src/main/java/ai/rapids/cudf/CuFile.java | 2 +- java/src/main/native/src/CuFileJni.cpp | 11 ++-- .../test/java/ai/rapids/cudf/CuFileTest.java | 57 ++++++++++++++++--- 4 files changed, 58 insertions(+), 14 deletions(-) diff --git a/java/pom.xml b/java/pom.xml index 387ef1cb65b..79d1125dfc0 100755 --- a/java/pom.xml +++ b/java/pom.xml @@ -330,7 +330,7 @@ org.junit.jupiter junit-jupiter-engine - 5.2.0 + 5.4.2 diff --git a/java/src/main/java/ai/rapids/cudf/CuFile.java b/java/src/main/java/ai/rapids/cudf/CuFile.java index d4be9a0194a..b1e752355db 100644 --- a/java/src/main/java/ai/rapids/cudf/CuFile.java +++ b/java/src/main/java/ai/rapids/cudf/CuFile.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2020, NVIDIA CORPORATION. + * Copyright (c) 2020-2021, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. diff --git a/java/src/main/native/src/CuFileJni.cpp b/java/src/main/native/src/CuFileJni.cpp index cd563bb4b29..054665dc714 100644 --- a/java/src/main/native/src/CuFileJni.cpp +++ b/java/src/main/native/src/CuFileJni.cpp @@ -1,5 +1,5 @@ /* - * Copyright (c) 2020, NVIDIA CORPORATION. + * Copyright (c) 2020-2021, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -20,6 +20,8 @@ #include #include +#include +#include #include "jni_utils.hpp" @@ -247,12 +249,13 @@ class cufile_file { * @return The file offset from which the buffer was appended. */ std::size_t append(cufile_buffer const &buffer) { - auto const status = lseek(file_descriptor_, 0, SEEK_END); + struct stat stat_buffer; + auto const status = fstat(file_descriptor_, &stat_buffer); if (status < 0) { - CUDF_FAIL("Failed to seek end of file: " + cuFileGetErrorString(errno)); + CUDF_FAIL("Failed to get file status for appending: " + cuFileGetErrorString(errno)); } - auto const file_offset = static_cast(status); + auto const file_offset = static_cast(stat_buffer.st_size); write(buffer, file_offset); return file_offset; } diff --git a/java/src/test/java/ai/rapids/cudf/CuFileTest.java b/java/src/test/java/ai/rapids/cudf/CuFileTest.java index 63ae13c00da..c07a8b85bdf 100644 --- a/java/src/test/java/ai/rapids/cudf/CuFileTest.java +++ b/java/src/test/java/ai/rapids/cudf/CuFileTest.java @@ -1,3 +1,19 @@ +/* + * Copyright (c) 2020-2021, 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 ai.rapids.cudf; import org.junit.jupiter.api.AfterEach; @@ -5,13 +21,12 @@ import org.junit.jupiter.api.io.TempDir; import java.io.File; +import java.io.IOException; -import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.*; import static org.junit.jupiter.api.Assumptions.assumeTrue; public class CuFileTest extends CudfTestBase { - @TempDir File tempDir; - @AfterEach void tearDown() { if (PinnedMemoryPool.isInitialized()) { @@ -20,9 +35,38 @@ void tearDown() { } @Test - public void testCopyToFile() { + public void testCopyToFile(@TempDir File tempDir) { + assumeTrue(CuFile.libraryLoaded()); + File tempFile = new File(tempDir, "tempFile"); + assertFalse(tempFile.exists()); + verifyCopyToFile(tempFile); + } + + @Test + public void testCopyToExistingFile(@TempDir File tempDir) throws IOException { + assumeTrue(CuFile.libraryLoaded()); + File tempFile = new File(tempDir, "tempFile"); + assertTrue(tempFile.createNewFile()); + verifyCopyToFile(tempFile); + } + + @Test + public void testAppendToFile(@TempDir File tempDir) { + assumeTrue(CuFile.libraryLoaded()); + File tempFile = new File(tempDir, "tempFile"); + assertFalse(tempFile.exists()); + verifyAppendToFile(tempFile); + } + + @Test + public void testAppendToExistingFile(@TempDir File tempDir) throws IOException { assumeTrue(CuFile.libraryLoaded()); File tempFile = new File(tempDir, "tempFile"); + assertTrue(tempFile.createNewFile()); + verifyAppendToFile(tempFile); + } + + private void verifyCopyToFile(File tempFile) { try (HostMemoryBuffer orig = HostMemoryBuffer.allocate(16); DeviceMemoryBuffer from = DeviceMemoryBuffer.allocate(16); DeviceMemoryBuffer to = DeviceMemoryBuffer.allocate(16); @@ -36,10 +80,7 @@ public void testCopyToFile() { } } - @Test - public void testAppendToFile() { - assumeTrue(CuFile.libraryLoaded()); - File tempFile = new File(tempDir, "tempFile"); + private void verifyAppendToFile(File tempFile) { try (HostMemoryBuffer orig = HostMemoryBuffer.allocate(16); DeviceMemoryBuffer from = DeviceMemoryBuffer.allocate(16); DeviceMemoryBuffer to = DeviceMemoryBuffer.allocate(16); From fc9a00f979759959cd71b093869ce9aa2c347ae7 Mon Sep 17 00:00:00 2001 From: Michael Wang Date: Thu, 4 Feb 2021 06:46:07 -0800 Subject: [PATCH 7/8] Improve `assert_eq` handling of scalar (#7220) Closes #7199 Refactors scalar handling inside `assert_eq`. On higher level, this PR proposes a "whitelist" style testing: all compares should go to the "strict equal" code path unless explicitly allowed. This allows the test system to capture all unintended inequality except the ones that's discussed upon. For example, this PR creates two whitelist items: - If the operands overrides `__eq__`, use it to determine equality. - If the operands are floating type, assert approximate equality. For all other cases, the operands should be strictly equal. Note that for testing purposes, `np.nan` are considered equal to itself. Authors: - Michael Wang (@isVoid) Approvers: - GALI PREM SAGAR (@galipremsagar) - @brandon-b-miller URL: https://github.com/rapidsai/cudf/pull/7220 --- python/cudf/cudf/tests/test_testing.py | 40 +++++++++++++++++++++++++- python/cudf/cudf/tests/utils.py | 8 +++--- 2 files changed, 43 insertions(+), 5 deletions(-) diff --git a/python/cudf/cudf/tests/test_testing.py b/python/cudf/cudf/tests/test_testing.py index 6fb4ae96da2..eee7078433d 100644 --- a/python/cudf/cudf/tests/test_testing.py +++ b/python/cudf/cudf/tests/test_testing.py @@ -10,7 +10,7 @@ assert_index_equal, assert_series_equal, ) -from cudf.tests.utils import NUMERIC_TYPES, OTHER_TYPES +from cudf.tests.utils import NUMERIC_TYPES, OTHER_TYPES, assert_eq @pytest.mark.parametrize("rdata", [[1, 2, 5], [1, 2, 6], [1, 2, 5, 6]]) @@ -299,3 +299,41 @@ def test_range_index_and_int_index_eqaulity(index, exact): assert_index_equal(idx1, idx2, exact=exact) else: assert_index_equal(idx1, idx2, exact=exact) + + +@pytest.mark.parametrize( + "left, right", + [ + (1493282, 1493282), + (1493282.0, 1493282.0 + 1e-8), + ("abc", "abc"), + (0, np.array(0)), + ( + np.datetime64(123456, "ns"), + pd.Timestamp(np.datetime64(123456, "ns")), + ), + ("int64", np.dtype("int64")), + (np.nan, np.nan), + ], +) +def test_basic_scalar_equality(left, right): + assert_eq(left, right) + + +@pytest.mark.parametrize( + "left, right", + [ + (1493282, 1493274), + (1493282.0, 1493282.0 + 1e-6), + ("abc", "abd"), + (0, np.array(1)), + ( + np.datetime64(123456, "ns"), + pd.Timestamp(np.datetime64(123457, "ns")), + ), + ("int64", np.dtype("int32")), + ], +) +def test_basic_scalar_inequality(left, right): + with pytest.raises(AssertionError, match=r".*not (almost )?equal.*"): + assert_eq(left, right) diff --git a/python/cudf/cudf/tests/utils.py b/python/cudf/cudf/tests/utils.py index cd6b9d54b21..ae60aad8db2 100644 --- a/python/cudf/cudf/tests/utils.py +++ b/python/cudf/cudf/tests/utils.py @@ -97,13 +97,13 @@ def assert_eq(left, right, **kwargs): else: assert np.array_equal(left, right) else: + # Use the overloaded __eq__ of the operands if left == right: return True + elif any([np.issubdtype(type(x), np.floating) for x in (left, right)]): + np.testing.assert_almost_equal(left, right) else: - if np.isnan(left): - assert np.isnan(right) - else: - assert np.allclose(left, right, equal_nan=True) + np.testing.assert_equal(left, right) return True From 568df5bd7d4c58ab3788f81126120124c0bf2304 Mon Sep 17 00:00:00 2001 From: GALI PREM SAGAR Date: Thu, 4 Feb 2021 08:59:25 -0600 Subject: [PATCH 8/8] Prepare Changelog for Automation (#7309) This PR prepares the changelog to be automatically updated during releases. Authors: - GALI PREM SAGAR (@galipremsagar) Approvers: - Keith Kraus (@kkraus14) - AJ Schmidt (@ajschmidt8) URL: https://github.com/rapidsai/cudf/pull/7309 --- CHANGELOG.md | 37 ------------------------------------- 1 file changed, 37 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 28d80c33f10..f712a700044 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,40 +1,3 @@ -# cuDF 0.18.0 (Date TBD) - -## New Features -- PR #6856 Add groupby idxmin, idxmax aggregation -- PR #6847 Add a cmake find module for cuFile in JNI code -- PR #6902 Implement `DataFrame.quantile` for `datetime` and `timedelta` data types -- PR #6814 Implement `cudf::reduce` for `decimal32` and `decimal64` (part 1) -- PR #6929 Add `Index.set_names` api -- PR #6907 Add `replace_null` API with `replace_policy` parameter, `fixed_width` column support -- PR #6885 Share `factorize` implementation with Index and cudf module -- PR #6775 Implement cudf.DateOffset for months -- PR #7069 Pack/unpack for serialization/deserialization of table_views -- PR #7039 Support contains() on lists of primitives - -## Improvements - -- PR #6938 Pass numeric scalars of the same dtype through numeric binops -- PR #6275 Update to official libcu++ on Github -- PR #6838 Fix `columns` & `index` handling in dataframe constructor -- PR #6750 Remove **kwargs from string/categorical methods -- PR #6585 Add dictionary support to libcudf groupby functions -- PR #6909 Support reading byte array backed decimal columns from parquet files -- PR #6939 Use simplified `rmm::exec_policy` -- PR #6512 Refactor rolling.cu to reduce compile time -- PR #6982 Disable some pragma unroll statements in thrust `sort.h` -- PR #7051 Verify decimal cast in java package -- PR #7120 Verify window operations on decimal in java package - -## Bug Fixes - -- PR #6884 Correct the sampling range when sampling with replacement -- PR #6903 Add null count test for apply_boolean_mask -- PR #6922 Fix N/A detection for empty fields in CSV reader -- PR #6912 Fix rmm_mode=managed parameter for gtests -- PR #6943 Fix join with nulls not equal performance -- PR #6945 Fix groupby agg/apply behaviour when no key columns are provided -- PR #6942 Fix cudf::merge gtest for dictionary columns # 0.18.0 Please see https://github.com/rapidsai/cudf/releases/tag/branch-0.18-latest for the latest changes to this development branch.