From 1a6f842bdfcb0b7591bd9b6433de3c7e629f5200 Mon Sep 17 00:00:00 2001 From: Ed Seidl Date: Tue, 12 Sep 2023 18:36:13 -0700 Subject: [PATCH 01/39] add decoder for DELTA_BYTE_ARRAY --- cpp/src/io/parquet/delta_binary.cuh | 22 + cpp/src/io/parquet/page_decode.cuh | 1 + cpp/src/io/parquet/page_delta_decode.cu | 436 +++++++++++++++++++ cpp/src/io/parquet/page_hdr.cu | 4 + cpp/src/io/parquet/page_string_decode.cu | 319 +++++++++++++- cpp/src/io/parquet/parquet_gpu.hpp | 42 +- cpp/src/io/parquet/reader_impl.cpp | 27 +- cpp/src/io/parquet/reader_impl_preprocess.cu | 1 + 8 files changed, 814 insertions(+), 38 deletions(-) diff --git a/cpp/src/io/parquet/delta_binary.cuh b/cpp/src/io/parquet/delta_binary.cuh index 4fc8b9cfb8e..d4b74d2b858 100644 --- a/cpp/src/io/parquet/delta_binary.cuh +++ b/cpp/src/io/parquet/delta_binary.cuh @@ -185,6 +185,28 @@ struct delta_binary_decoder { } } + // given start/end pointers in the data, find the end of the binary encoded block. when done, + // `this` will be initialized with the correct start and end positions. returns the end, which is + // start of data/next block. should only be called from thread 0. + inline __device__ uint8_t const* find_end_of_block(uint8_t const* start, uint8_t const* end) + { + // read block header + init_binary_block(start, end); + + // test for no encoded values. a single value will be in the block header. + if (value_count <= 1) { return block_start; } + + // read mini-block headers and skip over data + while (current_value_idx < num_encoded_values(false)) { + setup_next_mini_block(false); + } + // calculate the correct end of the block + auto const* const new_end = cur_mb == 0 ? block_start : cur_mb_start; + // re-init block with correct end + init_binary_block(start, new_end); + return new_end; + } + // decode the current mini-batch of deltas, and convert to values. // called by all threads in a warp, currently only one warp supported. inline __device__ void calc_mini_block_values(int lane_id) diff --git a/cpp/src/io/parquet/page_decode.cuh b/cpp/src/io/parquet/page_decode.cuh index 26e3c951b2e..e780256956f 100644 --- a/cpp/src/io/parquet/page_decode.cuh +++ b/cpp/src/io/parquet/page_decode.cuh @@ -1282,6 +1282,7 @@ inline __device__ bool setupLocalPageInfo(page_state_s* const s, s->dict_run = 0; } break; case Encoding::DELTA_BINARY_PACKED: + case Encoding::DELTA_BYTE_ARRAY: // nothing to do, just don't error break; default: diff --git a/cpp/src/io/parquet/page_delta_decode.cu b/cpp/src/io/parquet/page_delta_decode.cu index e79a479388f..a9e4cceade0 100644 --- a/cpp/src/io/parquet/page_delta_decode.cu +++ b/cpp/src/io/parquet/page_delta_decode.cu @@ -27,6 +27,275 @@ namespace cudf::io::parquet::gpu { namespace { +constexpr int decode_block_size = 128; + +struct delta_byte_array_decoder { + uint8_t const* last_string; + uint8_t const* suffix_data; + + // used when skipping values + uint8_t* temp_buf; + uint32_t start_val; // decoded strings up to this index will be dumped to temp_buf + uint32_t last_string_len; + + delta_binary_decoder prefixes; // state of decoder for prefix lengths + delta_binary_decoder suffixes; // state of decoder for suffix lengths + // size_type offset[non_zero_buffer_size]; // circular buffer for string output offsets + + // initialize the prefixes and suffixes blocks + __device__ void init(uint8_t const* start, uint8_t const* end, uint32_t start_idx, uint8_t* temp) + { + auto const* suffix_start = prefixes.find_end_of_block(start, end); + suffix_data = suffixes.find_end_of_block(suffix_start, end); + last_string = nullptr; + temp_buf = temp; + start_val = start_idx; + } + + // kind of like an inclusive scan for strings. takes prefix_len bytes from preceding + // string and prepends to the suffix we've already copied into place. called from + // within loop over values_in_mb, so this only needs to handle a single warp worth of data + // at a time. + __device__ void string_scan(uint8_t* strings_out, + uint8_t const* last_string, + uint32_t start_idx, + uint32_t end_idx, + uint32_t offset, + uint32_t lane_id) + { + using cudf::detail::warp_size; + + // let p(n) === length(prefix(string_n)) + // + // if p(n-1) > p(n), then string_n can be completed when string_n-2 is completed. likewise if + // p(m) > p(n), then string_n can be completed with string_m-1. however, if p(m) < p(n), then m + // is a "blocker" for string_n; string_n can be completed only after string_m is. + // + // we will calculate the nearest blocking position for each lane, and then fill in string_0. we + // then iterate, finding all lanes that have had their "blocker" filled in and completing them. + // when all lanes are filled in, we return. this will still hit the worst case if p(n-1) < p(n) + // for all n + __shared__ __align__(8) int64_t prefix_lens[warp_size]; + __shared__ __align__(8) uint8_t const* offsets[warp_size]; + + uint32_t const ln_idx = start_idx + lane_id; + uint32_t const src_idx = rolling_index(ln_idx); + uint64_t prefix_len = ln_idx < end_idx ? prefixes.value[src_idx] : 0; + uint8_t* const lane_out = ln_idx < end_idx ? strings_out + offset : nullptr; + + prefix_lens[lane_id] = prefix_len; + offsets[lane_id] = lane_out; + + // if all prefix_len's are zero, then there's nothing to do + if (__all_sync(0xffff'ffff, prefix_len == 0)) { return; } + + // find a neighbor to the left that has a prefix length less than this lane. once that + // neighbor is complete, this lane can be completed. + int blocker = lane_id - 1; + while (blocker > 0 && prefix_lens[blocker] != 0 && prefix_len <= prefix_lens[blocker]) { + blocker--; + } + + // fill in lane 0 (if necessary) + if (lane_id == 0 && prefix_len > 0) { + memcpy(lane_out, last_string, prefix_len); + prefix_lens[0] = prefix_len = 0; + } + __syncwarp(); + + // now fill in blockers until done + for (uint32_t i = 1; i < warp_size && i + start_idx < end_idx; i++) { + if (prefix_len != 0 && prefix_lens[blocker] == 0 && lane_out != nullptr) { + memcpy(lane_out, offsets[blocker], prefix_len); + prefix_lens[lane_id] = prefix_len = 0; + } + + // check for finished + if (__all_sync(0xffff'ffff, prefix_len == 0)) { return; } + } + } + + // calculate a mini-batch of string values, writing the results to + // `strings_out`. starting at global index `start_idx` and decoding + // up to `num_values` strings. + // called by all threads in a warp. used for strings < 32 chars. + // returns number of bytes written + __device__ size_t calculate_string_values(uint8_t* strings_out, + uint32_t start_idx, + uint32_t num_values, + uint32_t lane_id) + { + using cudf::detail::warp_size; + using WarpScan = cub::WarpScan; + __shared__ WarpScan::TempStorage scan_temp; + + if (start_idx >= suffixes.value_count) { return 0; } + auto end_idx = start_idx + min(suffixes.values_per_mb, num_values); + end_idx = min(end_idx, static_cast(suffixes.value_count)); + + auto p_strings_out = strings_out; + auto p_temp_out = temp_buf; + + auto copy_batch = [&](uint8_t* out, uint32_t idx, uint32_t end) { + uint32_t ln_idx = idx + lane_id; + + // calculate offsets into suffix data + uint32_t const src_idx = rolling_index(ln_idx); + uint64_t const suffix_len = ln_idx < end ? suffixes.value[src_idx] : 0; + uint64_t suffix_off = 0; + WarpScan(scan_temp).ExclusiveSum(suffix_len, suffix_off); + + // calculate offsets into string data and save in string_offsets + uint64_t const prefix_len = ln_idx < end ? prefixes.value[src_idx] : 0; + uint64_t const string_len = prefix_len + suffix_len; + + // get offset into output for each lane + uint64_t string_off, warp_total; + WarpScan(scan_temp).ExclusiveSum(string_len, string_off, warp_total); + auto const so_ptr = out + string_off; + + // copy suffixes into string data + if (ln_idx < end) { memcpy(so_ptr + prefix_len, suffix_data + suffix_off, suffix_len); } + __syncwarp(); + + // copy prefixes into string data. + string_scan(out, last_string, idx, end, string_off, lane_id); + + // save the position of the last computed string. this will be used in + // the next iteration to reconstruct the string in lane 0. + if (ln_idx == end - 1 || (ln_idx < end && lane_id == 31)) { + // set last_string to this lane's string + last_string = out + string_off; + last_string_len = string_len; + // and consume used suffix_data + suffix_data += suffix_off + suffix_len; + } + + return warp_total; + }; + + uint64_t string_total = 0; + for (int idx = start_idx; idx < end_idx; idx += warp_size) { + auto const n_in_batch = min(warp_size, end_idx - idx); + // account for the case where start_val occurs in the middle of this batch + if (idx < start_val && idx + n_in_batch > start_val) { + // dump idx...start_val into temp_buf + copy_batch(p_temp_out, idx, start_val); + __syncwarp(); + + // start_val...idx + n_in_batch into strings_out + auto nbytes = copy_batch(p_strings_out, start_val, idx + n_in_batch); + p_strings_out += nbytes; + string_total = nbytes; + } else { + if (idx < start_val) { + p_temp_out += copy_batch(p_temp_out, idx, end_idx); + } else { + auto nbytes = copy_batch(p_strings_out, idx, end_idx); + p_strings_out += nbytes; + string_total += nbytes; + } + } + __syncwarp(); + } + + return string_total; + } + + // character parallel version of CalculateStringValues(). This is faster for strings longer than + // 32 chars. + __device__ size_t calculate_string_values_cp(uint8_t* strings_out, + uint32_t start_idx, + uint32_t num_values, + uint32_t lane_id) + { + using cudf::detail::warp_size; + __shared__ __align__(8) uint8_t* so_ptr; + + if (start_idx >= suffixes.value_count) { return; } + auto end_idx = start_idx + min(suffixes.values_per_mb, num_values); + end_idx = min(end_idx, static_cast(suffixes.value_count)); + + if (lane_id == 0) { so_ptr = start_idx < start_val ? temp_buf : strings_out; } + __syncwarp(); + + uint64_t string_total = 0; + for (int idx = start_idx; idx < end_idx; idx++) { + uint32_t const src_idx = rolling_index(idx); + uint64_t const suffix_len = suffixes.value[src_idx]; + uint64_t const prefix_len = prefixes.value[src_idx]; + uint64_t const string_len = prefix_len + suffix_len; + + // copy prefix and suffix data into current strings_out position + // for longer strings use a 4-byte version stolen from gather_chars_fn_string_parallel. + if (string_len > 64) { + if (prefix_len > 0) { wideStrcpy(so_ptr, last_string, prefix_len, lane_id); } + if (suffix_len > 0) { wideStrcpy(so_ptr + prefix_len, suffix_data, suffix_len, lane_id); } + } else { + for (int i = lane_id; i < string_len; i += warp_size) { + so_ptr[i] = i < prefix_len ? last_string[i] : suffix_data[i - prefix_len]; + } + } + __syncwarp(); + + if (idx >= start_val) { string_total += string_len; } + + if (lane_id == 0) { + last_string = so_ptr; + last_string_len = string_len; + suffix_data += suffix_len; + if (idx == start_val - 1) { + so_ptr = strings_out; + } else { + so_ptr += string_len; + } + } + __syncwarp(); + } + + return string_total; + } + + // dump strings before start_val to temp buf + __device__ void skip(bool use_char_ll) + { + using cudf::detail::warp_size; + int const t = threadIdx.x; + int const lane_id = t % warp_size; + + // is this even necessary? return if asking to skip the whole block. + if (start_val >= prefixes.num_encoded_values(true)) { return; } + + // TODO: this assumes prefixes and suffixes will have the same parameters + auto const batch_size = prefixes.values_per_mb; + + uint32_t skip_pos = 0; + while (prefixes.current_value_idx < start_val) { + // warp 0 gets prefixes and warp 1 gets suffixes + auto* const db = t < 32 ? &prefixes : &suffixes; + + // this will potentially decode past start_val, but that's ok + if (t < 64) { db->decode_batch(); } + __syncthreads(); + + // warp 0 decodes the batch. + if (t < 32) { + auto const num_to_decode = min(batch_size, start_val - skip_pos); + auto const bytes_written = + use_char_ll ? calculate_string_values_cp(temp_buf, skip_pos, num_to_decode, lane_id) + : calculate_string_values(temp_buf, skip_pos, num_to_decode, lane_id); + // store last_string someplace safe in temp buffer + if (t == 0) { + memcpy(temp_buf + bytes_written, last_string, last_string_len); + last_string = temp_buf + bytes_written; + } + } + skip_pos += prefixes.values_per_mb; + __syncthreads(); + } + } +}; + // Decode page data that is DELTA_BINARY_PACKED encoded. This encoding is // only used for int32 and int64 physical types (and appears to only be used // with V2 page headers; see https://www.mail-archive.com/dev@parquet.apache.org/msg11826.html). @@ -147,6 +416,149 @@ __global__ void __launch_bounds__(96) gpuDecodeDeltaBinary( } } +// Decode page data that is DELTA_BYTE_ARRAY packed. This encoding consists of a DELTA_BINARY_PACKED +// array of prefix lengths, followed by a DELTA_BINARY_PACKED array of suffix lengths, followed by +// the suffixes (technically the suffixes are DELTA_LENGTH_BYTE_ARRAY encoded). The latter two can +// be used to create an offsets array for the suffix data, but then this needs to be combined with +// the prefix lengths to do the final decode for each value. Because the lengths of the prefixes and +// suffixes are not encoded in the header, we're going to have to first do a quick pass through them +// to find the start/end of each structure. +template +__global__ void __launch_bounds__(decode_block_size) gpuDecodeDeltaByteArray( + PageInfo* pages, device_span chunks, size_t min_row, size_t num_rows) +{ + using cudf::detail::warp_size; + __shared__ __align__(16) delta_byte_array_decoder db_state; + __shared__ __align__(16) page_state_s state_g; + __shared__ __align__(16) page_state_buffers_s state_buffers; + + page_state_s* const s = &state_g; + auto* const sb = &state_buffers; + int const page_idx = blockIdx.x; + int const t = threadIdx.x; + int const lane_id = t % warp_size; + auto* const prefix_db = &db_state.prefixes; + auto* const suffix_db = &db_state.suffixes; + auto* const dba = &db_state; + [[maybe_unused]] null_count_back_copier _{s, t}; + + if (!setupLocalPageInfo(s, + &pages[page_idx], + chunks, + min_row, + num_rows, + mask_filter{KERNEL_MASK_DELTA_BYTE_ARRAY}, + true)) { + return; + } + + bool const has_repetition = s->col.max_level[level_type::REPETITION] > 0; + + // TODO(ets) assert string_data != nullptr + + // choose a character parallel string copy when the average string is longer than a warp + auto const use_char_ll = (s->page.str_bytes / s->page.num_valids) > cudf::detail::warp_size; + + // copying logic from gpuDecodePageData. + PageNestingDecodeInfo const* nesting_info_base = s->nesting_info; + + __shared__ level_t rep[delta_rolling_buf_size]; // circular buffer of repetition level values + __shared__ level_t def[delta_rolling_buf_size]; // circular buffer of definition level values + + // skipped_leaf_values will always be 0 for flat hierarchies. + uint32_t const skipped_leaf_values = s->page.skipped_leaf_values; + + if (t == 0) { + // initialize the prefixes and suffixes blocks + dba->init(s->data_start, s->data_end, s->page.start_val, s->page.temp_string_buf); + } + __syncthreads(); + + // pointer to location to output final strings + int const leaf_level_index = s->col.max_nesting_depth - 1; + auto strings_data = nesting_info_base[leaf_level_index].string_out; + + // TODO(ets) assert that prefix and suffix have same mini-block size + auto const batch_size = prefix_db->values_per_mb; + + // if this is a bounds page and nested, then we need to skip up front. non-nested will work + // its way through the page. + int string_pos = has_repetition ? s->page.start_val : 0; + auto const is_bounds_pg = is_bounds_page(s, min_row, num_rows, has_repetition); + if (is_bounds_pg && string_pos > 0) { dba->skip(use_char_ll); } + + while (!s->error && (s->input_value_count < s->num_input_values || s->src_pos < s->nz_count)) { + uint32_t target_pos; + uint32_t const src_pos = s->src_pos; + + if (t < 96) { // warp 0..2 + target_pos = min(src_pos + 2 * (batch_size), s->nz_count + s->first_row + batch_size); + } else { // warp 3 + target_pos = min(s->nz_count, src_pos + batch_size); + } + __syncthreads(); + + // warp0 will decode the rep/def levels, warp1 will unpack a mini-batch of prefixes, warp 2 will + // unpack a mini-batch of suffixes. warp3 waits one cycle for warps 0-2 to produce a batch, and + // then stuffs values into the proper location in the output. + if (t < 32) { + // decode repetition and definition levels. + // - update validity vectors + // - updates offsets (for nested columns) + // - produces non-NULL value indices in s->nz_idx for subsequent decoding + gpuDecodeLevels(s, sb, target_pos, rep, def, t); + + } else if (t < 64) { + // warp 1 + prefix_db->decode_batch(); + + } else if (t < 96) { + // warp 2 + suffix_db->decode_batch(); + + } else if (src_pos < target_pos) { + // warp 3 + + int nproc = min(batch_size, s->page.end_val - string_pos); + strings_data += use_char_ll + ? dba->calculate_string_values_cp(strings_data, string_pos, nproc, lane_id) + : dba->calculate_string_values(strings_data, string_pos, nproc, lane_id); + string_pos += nproc; + + // process the mini-block in batches of 32 + for (uint32_t sp = src_pos + lane_id; sp < src_pos + batch_size; sp += 32) { + // the position in the output column/buffer + int dst_pos = sb->nz_idx[rolling_index(sp)]; + + // handle skip_rows here. flat hierarchies can just skip up to first_row. + if (!has_repetition) { dst_pos -= s->first_row; } + + if (dst_pos >= 0 && sp < target_pos) { + auto offptr = + reinterpret_cast(nesting_info_base[leaf_level_index].data_out) + dst_pos; + auto const src_idx = rolling_index(sp + skipped_leaf_values); + *offptr = prefix_db->value[src_idx] + suffix_db->value[src_idx]; + } + __syncwarp(); + } + + if (lane_id == 0) { s->src_pos = src_pos + batch_size; } + } + + __syncthreads(); + } + + // now turn array of lengths into offsets + int value_count = nesting_info_base[leaf_level_index].value_count; + + // if no repetition we haven't calculated start/end bounds and instead just skipped + // values until we reach first_row. account for that here. + if (!has_repetition) { value_count -= s->first_row; } + + auto const offptr = reinterpret_cast(nesting_info_base[leaf_level_index].data_out); + block_excl_sum(offptr, value_count, s->page.str_offset); +} + } // anonymous namespace /** @@ -173,4 +585,28 @@ void __host__ DecodeDeltaBinary(cudf::detail::hostdevice_vector& pages } } +/** + * @copydoc cudf::io::parquet::gpu::DecodeDeltaByteArray + */ +void __host__ DecodeDeltaByteArray(cudf::detail::hostdevice_vector& pages, + cudf::detail::hostdevice_vector const& chunks, + size_t num_rows, + size_t min_row, + int level_type_size, + rmm::cuda_stream_view stream) +{ + CUDF_EXPECTS(pages.size() > 0, "There is no page to decode"); + + dim3 dim_block(decode_block_size, 1); + dim3 dim_grid(pages.size(), 1); // 1 threadblock per page + + if (level_type_size == 1) { + gpuDecodeDeltaByteArray + <<>>(pages.device_ptr(), chunks, min_row, num_rows); + } else { + gpuDecodeDeltaByteArray + <<>>(pages.device_ptr(), chunks, min_row, num_rows); + } +} + } // namespace cudf::io::parquet::gpu diff --git a/cpp/src/io/parquet/page_hdr.cu b/cpp/src/io/parquet/page_hdr.cu index 0d611643b46..921987b9d40 100644 --- a/cpp/src/io/parquet/page_hdr.cu +++ b/cpp/src/io/parquet/page_hdr.cu @@ -168,6 +168,8 @@ __device__ uint32_t kernel_mask_for_page(gpu::PageInfo const& page, if (page.encoding == Encoding::DELTA_BINARY_PACKED) { return KERNEL_MASK_DELTA_BINARY; + } else if (page.encoding == Encoding::DELTA_BYTE_ARRAY) { + return KERNEL_MASK_DELTA_BYTE_ARRAY; } else if (is_string_col(chunk)) { return KERNEL_MASK_STRING; } @@ -392,6 +394,8 @@ __global__ void __launch_bounds__(128) bs->page.skipped_values = -1; bs->page.skipped_leaf_values = 0; bs->page.str_bytes = 0; + bs->page.temp_string_size = 0; + bs->page.temp_string_buf = nullptr; bs->page.kernel_mask = 0; } num_values = bs->ck.num_values; diff --git a/cpp/src/io/parquet/page_string_decode.cu b/cpp/src/io/parquet/page_string_decode.cu index 1ac4c95f713..1e1ae0a38f7 100644 --- a/cpp/src/io/parquet/page_string_decode.cu +++ b/cpp/src/io/parquet/page_string_decode.cu @@ -14,12 +14,19 @@ * limitations under the License. */ +#include "delta_binary.cuh" #include "page_decode.cuh" #include "page_string_utils.cuh" #include +#include #include +#include +#include + +#include + namespace cudf { namespace io { namespace parquet { @@ -27,10 +34,11 @@ namespace gpu { namespace { -constexpr int preprocess_block_size = 512; -constexpr int decode_block_size = 128; -constexpr int rolling_buf_size = decode_block_size * 2; -constexpr int preproc_buf_size = LEVEL_DECODE_BUF_SIZE; +constexpr int preprocess_block_size = 512; +constexpr int decode_block_size = 128; +constexpr int delta_preproc_block_size = 64; +constexpr int rolling_buf_size = decode_block_size * 2; +constexpr int preproc_buf_size = LEVEL_DECODE_BUF_SIZE; /** * @brief Compute the start and end page value bounds for this page @@ -452,6 +460,95 @@ __device__ size_t totalPlainEntriesSize(uint8_t const* data, return total_len; } +/** + * @brief Compute string size information for DELTA_BYTE_ARRAY encoded strings. + * + * Called with 64 threads + * + * @param data Pointer to the start of the page data stream + * @param end Pointer to the end of the page data stream + * @param start_value Do not count values that occur before this index + * @param end_value Do not count values that occur after this index + */ +__device__ thrust::pair totalDeltaByteArraySize(uint8_t const* data, + uint8_t const* end, + int start_value, + int end_value) +{ + using cudf::detail::warp_size; + using WarpReduce = cub::WarpReduce; + __shared__ typename WarpReduce::TempStorage temp_storage[2]; + + __shared__ __align__(16) delta_binary_decoder prefixes; + __shared__ __align__(16) delta_binary_decoder suffixes; + + int const t = threadIdx.x; + int const lane_id = t % warp_size; + int const warp_id = t / warp_size; + + if (t == 0) { + auto const* suffix_start = prefixes.find_end_of_block(data, end); + suffixes.init_binary_block(suffix_start, end); + } + __syncthreads(); + + // two warps will traverse the prefixes and suffixes and sum them up + auto const db = t < warp_size ? &prefixes : t < 2 * warp_size ? &suffixes : nullptr; + + size_t total_bytes = 0; + uleb128_t max_len = 0; + + if (db != nullptr) { + // initialize with first value (which is stored in last_value) + if (lane_id == 0 && start_value == 0) { total_bytes = db->last_value; } + + uleb128_t lane_sum = 0; + uleb128_t lane_max = 0; + while (db->current_value_idx < end_value && + db->current_value_idx < db->num_encoded_values(true)) { + // calculate values for current mini-block + db->calc_mini_block_values(lane_id); + + // get per lane sum for mini-block + for (uint32_t i = 0; i < db->values_per_mb; i += 32) { + uint32_t const idx = db->current_value_idx + i + lane_id; + if (idx >= start_value && idx < end_value && idx < db->value_count) { + lane_sum += db->value[rolling_index(idx)]; + lane_max = max(lane_max, db->value[rolling_index(idx)]); + } + } + + if (lane_id == 0) { db->setup_next_mini_block(true); } + __syncwarp(); + } + + // get sum for warp. + // note: warp_sum will only be valid on lane 0. + using cudf::detail::warp_size; + auto const warp_sum = WarpReduce(temp_storage[warp_id]).Sum(lane_sum); + auto const warp_max = WarpReduce(temp_storage[warp_id]).Reduce(lane_max, cub::Max()); + + if (lane_id == 0) { + total_bytes += warp_sum; + max_len = warp_max; + } + } + __syncthreads(); + + // now sum up total_bytes from the two warps + auto const final_bytes = + cudf::detail::single_lane_block_sum_reduce(total_bytes); + + // sum up prefix and suffix max lengths + auto final_max = cudf::detail::single_lane_block_sum_reduce(max_len); + if (t == 0) { + // save enough for one mimi-block plus some extra to save the last_string + final_max *= db->values_per_mb + 1; + } + + return {final_bytes, final_max}; +} + /** * @brief Kernel for computing string page output size information. * @@ -467,7 +564,7 @@ __device__ size_t totalPlainEntriesSize(uint8_t const* data, * @tparam level_t Type used to store decoded repetition and definition levels */ template -__global__ void __launch_bounds__(preprocess_block_size) gpuComputePageStringSizes( +__global__ void __launch_bounds__(preprocess_block_size) gpuComputeStringPageBounds( PageInfo* pages, device_span chunks, size_t min_row, size_t num_rows) { __shared__ __align__(16) page_state_s state_g; @@ -477,8 +574,12 @@ __global__ void __launch_bounds__(preprocess_block_size) gpuComputePageStringSiz int const t = threadIdx.x; PageInfo* const pp = &pages[page_idx]; - // reset str_bytes to 0 in case it's already been calculated - if (t == 0) { pp->str_bytes = 0; } + if (t == 0) { + s->page.num_nulls = 0; + s->page.num_valids = 0; + // reset str_bytes to 0 in case it's already been calculated + pp->str_bytes = 0; + } // whether or not we have repetition levels (lists) bool const has_repetition = chunks[pp->chunk_idx].max_level[level_type::REPETITION] > 0; @@ -492,25 +593,19 @@ __global__ void __launch_bounds__(preprocess_block_size) gpuComputePageStringSiz __shared__ rle_run rep_runs[rle_run_buffer_size]; rle_stream decoders[level_type::NUM_LEVEL_TYPES] = {{def_runs}, {rep_runs}}; - // setup page info - if (!setupLocalPageInfo( - s, pp, chunks, min_row, num_rows, mask_filter{KERNEL_MASK_STRING}, false)) { + if (!setupLocalPageInfo(s, + pp, + chunks, + min_row, + num_rows, + mask_filter{KERNEL_MASK_STRING | KERNEL_MASK_DELTA_BYTE_ARRAY}, + true)) { return; } - if (!t) { - s->page.num_nulls = 0; - s->page.num_valids = 0; - s->page.str_bytes = 0; - } - __syncthreads(); - bool const is_bounds_pg = is_bounds_page(s, min_row, num_rows, has_repetition); - // if we're skipping this page anyway, no need to count it - if (!is_bounds_pg && !is_page_contained(s, min_row, num_rows)) { return; } - // find start/end value indices auto const [start_value, end_value] = page_bounds(s, min_row, num_rows, is_bounds_pg, has_repetition, decoders); @@ -519,7 +614,111 @@ __global__ void __launch_bounds__(preprocess_block_size) gpuComputePageStringSiz if (t == 0) { pp->num_nulls = s->page.num_nulls; pp->num_valids = s->page.num_valids; + pp->start_val = start_value; + pp->end_val = end_value; } +} + +/** + * @brief Kernel for computing string page output size information for delta_byte_array encoding. + * + * String columns need accurate data size information to preallocate memory in the column buffer to + * store the char data. This calls a kernel to calculate information needed by the string decoding + * kernel. On exit, the `str_bytes`, `num_nulls`, and `num_valids` fields of the PageInfo struct + * are updated. This call ignores non-string columns. + * + * @param pages All pages to be decoded + * @param chunks All chunks to be decoded + * @param min_rows crop all rows below min_row + * @param num_rows Maximum number of rows to read + */ +__global__ void __launch_bounds__(delta_preproc_block_size) gpuComputeDeltaPageStringSizes( + PageInfo* pages, device_span chunks, size_t min_row, size_t num_rows) +{ + __shared__ __align__(16) page_state_s state_g; + + page_state_s* const s = &state_g; + int const page_idx = blockIdx.x; + int const t = threadIdx.x; + PageInfo* const pp = &pages[page_idx]; + + // whether or not we have repetition levels (lists) + bool const has_repetition = chunks[pp->chunk_idx].max_level[level_type::REPETITION] > 0; + + // setup page info + if (!setupLocalPageInfo( + s, pp, chunks, min_row, num_rows, mask_filter{KERNEL_MASK_DELTA_BYTE_ARRAY}, true)) { + return; + } + + auto const start_value = pp->start_val; + + // if data size is known, can short circuit here + if ((chunks[pp->chunk_idx].data_type & 7) == FIXED_LEN_BYTE_ARRAY) { + if (t == 0) { + pp->str_bytes = pp->num_valids * s->dtype_len_in; + + // only need temp space if we're skipping values + if (start_value > 0) { + // just need to parse the header of the first delta binary block to get values_per_mb + delta_binary_decoder db; + db.init_binary_block(s->data_start, s->data_end); + // save enough for one mimi-block plus some extra to save the last_string + pp->temp_string_size = s->dtype_len_in * (db.values_per_mb + 1); + } + } + } else { + // now process string info in the range [start_value, end_value) + // set up for decoding strings...can be either plain or dictionary + uint8_t const* data = s->data_start; + uint8_t const* const end = s->data_end; + auto const end_value = pp->end_val; + + auto const [len, max] = totalDeltaByteArraySize(data, end, start_value, end_value); + + if (t == 0) { + // TODO check for overflow + pp->str_bytes = len; + + // only need temp space if we're skipping values + if (start_value > 0) { pp->temp_string_size = max; } + } + } +} + +/** + * @brief Kernel for computing string page output size information. + * + * String columns need accurate data size information to preallocate memory in the column buffer to + * store the char data. This calls a kernel to calculate information needed by the string decoding + * kernel. On exit, the `str_bytes`, `num_nulls`, and `num_valids` fields of the PageInfo struct + * are updated. This call ignores non-string columns. + * + * @param pages All pages to be decoded + * @param chunks All chunks to be decoded + * @param min_rows crop all rows below min_row + * @param num_rows Maximum number of rows to read + */ +__global__ void __launch_bounds__(preprocess_block_size) gpuComputePageStringSizes( + PageInfo* pages, device_span chunks, size_t min_row, size_t num_rows) +{ + __shared__ __align__(16) page_state_s state_g; + + page_state_s* const s = &state_g; + int const page_idx = blockIdx.x; + int const t = threadIdx.x; + PageInfo* const pp = &pages[page_idx]; + + // whether or not we have repetition levels (lists) + bool const has_repetition = chunks[pp->chunk_idx].max_level[level_type::REPETITION] > 0; + + // setup page info + if (!setupLocalPageInfo( + s, pp, chunks, min_row, num_rows, mask_filter{KERNEL_MASK_STRING}, true)) { + return; + } + + bool const is_bounds_pg = is_bounds_page(s, min_row, num_rows, has_repetition); auto const& col = s->col; size_t str_bytes = 0; @@ -533,6 +732,8 @@ __global__ void __launch_bounds__(preprocess_block_size) gpuComputePageStringSiz uint8_t const* const end = s->data_end; uint8_t const* dict_base = nullptr; int dict_size = 0; + auto const start_value = pp->start_val; + auto const end_value = pp->end_val; switch (pp->encoding) { case Encoding::PLAIN_DICTIONARY: @@ -564,6 +765,9 @@ __global__ void __launch_bounds__(preprocess_block_size) gpuComputePageStringSiz if (t == 0) { // TODO check for overflow pp->str_bytes = str_bytes; + + // only need temp space for delta + pp->temp_string_size = 0; } } @@ -585,6 +789,7 @@ template __global__ void __launch_bounds__(decode_block_size) gpuDecodeStringPageData( PageInfo* pages, device_span chunks, size_t min_row, size_t num_rows) { + using cudf::detail::warp_size; __shared__ __align__(16) page_state_s state_g; __shared__ __align__(4) size_type last_offset; __shared__ __align__(16) @@ -595,6 +800,7 @@ __global__ void __launch_bounds__(decode_block_size) gpuDecodeStringPageData( auto* const sb = &state_buffers; int const page_idx = blockIdx.x; int const t = threadIdx.x; + int const lane_id = t % warp_size; [[maybe_unused]] null_count_back_copier _{s, t}; if (!setupLocalPageInfo( @@ -641,9 +847,9 @@ __global__ void __launch_bounds__(decode_block_size) gpuDecodeStringPageData( // WARP1: Decode dictionary indices, booleans or string positions if (s->dict_base) { - src_target_pos = gpuDecodeDictionaryIndices(s, sb, src_target_pos, t & 0x1f).first; + src_target_pos = gpuDecodeDictionaryIndices(s, sb, src_target_pos, lane_id).first; } else { - gpuInitStringDescriptors(s, sb, src_target_pos, t & 0x1f); + gpuInitStringDescriptors(s, sb, src_target_pos, lane_id); } if (t == 32) { *(volatile int32_t*)&s->dict_pos = src_target_pos; } } else { @@ -744,6 +950,16 @@ __global__ void __launch_bounds__(decode_block_size) gpuDecodeStringPageData( block_excl_sum(offptr, value_count, s->page.str_offset); } +struct page_tform_functor { + void* const data; + + __device__ PageInfo operator()(PageInfo& page, int64_t offset) + { + if (page.temp_string_size != 0) { page.temp_string_buf = static_cast(data) + offset; } + return page; + } +}; + } // anonymous namespace /** @@ -751,20 +967,75 @@ __global__ void __launch_bounds__(decode_block_size) gpuDecodeStringPageData( */ void ComputePageStringSizes(cudf::detail::hostdevice_vector& pages, cudf::detail::hostdevice_vector const& chunks, + rmm::device_buffer& temp_string_buf, size_t min_row, size_t num_rows, int level_type_size, + uint32_t kernel_mask, rmm::cuda_stream_view stream) { dim3 dim_block(preprocess_block_size, 1); dim3 dim_grid(pages.size(), 1); // 1 threadblock per page if (level_type_size == 1) { - gpuComputePageStringSizes + gpuComputeStringPageBounds <<>>(pages.device_ptr(), chunks, min_row, num_rows); } else { - gpuComputePageStringSizes + gpuComputeStringPageBounds <<>>(pages.device_ptr(), chunks, min_row, num_rows); } + + // kernel mask may contain other kernels we don't need to count + int count_mask = kernel_mask & (KERNEL_MASK_DELTA_BYTE_ARRAY | KERNEL_MASK_STRING); + int nkernels = std::bitset<32>(count_mask).count(); + auto streams = cudf::detail::fork_streams(stream, nkernels); + + int s_idx = 0; + if ((kernel_mask & KERNEL_MASK_DELTA_BYTE_ARRAY) != 0) { + dim3 dim_delta(delta_preproc_block_size, 1); + gpuComputeDeltaPageStringSizes<<>>( + pages.device_ptr(), chunks, min_row, num_rows); + } + if ((kernel_mask & KERNEL_MASK_STRING) != 0) { + gpuComputePageStringSizes<<>>( + pages.device_ptr(), chunks, min_row, num_rows); + } + + // synchronize the streams + cudf::detail::join_streams(streams, stream); + + // check for needed temp space for DELTA_BYTE_ARRAY + auto const need_sizes = thrust::any_of( + rmm::exec_policy(stream), pages.d_begin(), pages.d_end(), [] __device__(auto& page) { + return page.temp_string_size != 0; + }); + + if (need_sizes) { + auto const page_sizes = [] __device__(PageInfo const& page) { return page.temp_string_size; }; + int64_t total_size = thrust::transform_reduce(rmm::exec_policy(stream), + pages.d_begin(), + pages.d_end(), + page_sizes, + 0L, + thrust::plus{}); + + rmm::device_uvector page_string_offsets(pages.size(), stream); + thrust::transform_exclusive_scan(rmm::exec_policy(stream), + pages.d_begin(), + pages.d_end(), + page_string_offsets.begin(), + page_sizes, + 0L, + thrust::plus{}); + + temp_string_buf = rmm::device_buffer(total_size, stream); + + thrust::transform(rmm::exec_policy(stream), + pages.d_begin(), + pages.d_end(), + page_string_offsets.begin(), + pages.d_begin(), + page_tform_functor{temp_string_buf.data()}); + } } /** diff --git a/cpp/src/io/parquet/parquet_gpu.hpp b/cpp/src/io/parquet/parquet_gpu.hpp index e82b6abc13d..d6a872ac3a8 100644 --- a/cpp/src/io/parquet/parquet_gpu.hpp +++ b/cpp/src/io/parquet/parquet_gpu.hpp @@ -27,6 +27,7 @@ #include #include +#include #include #include #include @@ -99,9 +100,10 @@ enum level_type { * Used to control which decode kernels to run. */ enum kernel_mask_bits { - KERNEL_MASK_GENERAL = (1 << 0), // Run catch-all decode kernel - KERNEL_MASK_STRING = (1 << 1), // Run decode kernel for string data - KERNEL_MASK_DELTA_BINARY = (1 << 2) // Run decode kernel for DELTA_BINARY_PACKED data + KERNEL_MASK_GENERAL = (1 << 0), // Run catch-all decode kernel + KERNEL_MASK_STRING = (1 << 1), // Run decode kernel for string data + KERNEL_MASK_DELTA_BINARY = (1 << 2), // Run decode kernel for DELTA_BINARY_PACKED data + KERNEL_MASK_DELTA_BYTE_ARRAY = (1 << 3) // Run decode kernel for DELTA_BYTE_ARRAY encoded data }; /** @@ -182,9 +184,11 @@ struct PageInfo { int32_t num_input_values; int32_t chunk_row; // starting row of this page relative to the start of the chunk int32_t num_rows; // number of rows in this page - // the next two are calculated in gpuComputePageStringSizes + // the next four are calculated in gpuComputePageStringSizes int32_t num_nulls; // number of null values (V2 header), but recalculated for string cols int32_t num_valids; // number of non-null values, taking into account skip_rows/num_rows + int32_t start_val; // index of first value of the string data stream to use + int32_t end_val; // index of last value in string data stream int32_t chunk_idx; // column chunk this page belongs to int32_t src_col_schema; // schema index of this column uint8_t flags; // PAGEINFO_FLAGS_XXX @@ -221,6 +225,10 @@ struct PageInfo { // level decode buffers uint8_t* lvl_decode_buf[level_type::NUM_LEVEL_TYPES]; + // temporary space for decoding DELTA_BYTE_ARRAY encoded strings + int64_t temp_string_size; + uint8_t* temp_string_buf; + uint32_t kernel_mask; }; @@ -543,16 +551,20 @@ void ComputePageSizes(cudf::detail::hostdevice_vector& pages, * * @param[in,out] pages All pages to be decoded * @param[in] chunks All chunks to be decoded + * @param[out] temp_string_buf Temporary space needed for decoding DELTA_BYTE_ARRAY strings * @param[in] min_rows crop all rows below min_row * @param[in] num_rows Maximum number of rows to read * @param[in] level_type_size Size in bytes of the type for level decoding + * @param[in] kernel_mask Mask of kernels to run * @param[in] stream CUDA stream to use */ void ComputePageStringSizes(cudf::detail::hostdevice_vector& pages, cudf::detail::hostdevice_vector const& chunks, + rmm::device_buffer& temp_string_buf, size_t min_row, size_t num_rows, int level_type_size, + uint32_t kernel_mask, rmm::cuda_stream_view stream); /** @@ -606,7 +618,7 @@ void DecodeStringPageData(cudf::detail::hostdevice_vector& pages, * @param[in] num_rows Total number of rows to read * @param[in] min_row Minimum number of rows to read * @param[in] level_type_size Size in bytes of the type for level decoding - * @param[in] stream CUDA stream to use, default 0 + * @param[in] stream CUDA stream to use */ void DecodeDeltaBinary(cudf::detail::hostdevice_vector& pages, cudf::detail::hostdevice_vector const& chunks, @@ -615,6 +627,26 @@ void DecodeDeltaBinary(cudf::detail::hostdevice_vector& pages, int level_type_size, rmm::cuda_stream_view stream); +/** + * @brief Launches kernel for reading the DELTA_BYTE_ARRAY column data stored in the pages + * + * The page data will be written to the output pointed to in the page's + * associated column chunk. + * + * @param[in,out] pages All pages to be decoded + * @param[in] chunks All chunks to be decoded + * @param[in] num_rows Total number of rows to read + * @param[in] min_row Minimum number of rows to read + * @param[in] level_type_size Size in bytes of the type for level decoding + * @param[in] stream CUDA stream to use + */ +void DecodeDeltaByteArray(cudf::detail::hostdevice_vector& pages, + cudf::detail::hostdevice_vector const& chunks, + size_t num_rows, + size_t min_row, + int level_type_size, + rmm::cuda_stream_view stream); + /** * @brief Launches kernel for initializing encoder row group fragments * diff --git a/cpp/src/io/parquet/reader_impl.cpp b/cpp/src/io/parquet/reader_impl.cpp index 8b0a0bd4eb0..f062ce9832d 100644 --- a/cpp/src/io/parquet/reader_impl.cpp +++ b/cpp/src/io/parquet/reader_impl.cpp @@ -20,7 +20,6 @@ #include #include #include -#include #include #include @@ -29,10 +28,15 @@ namespace cudf::io::detail::parquet { void reader::impl::decode_page_data(size_t skip_rows, size_t num_rows) { - auto& chunks = _file_itm_data.chunks; - auto& pages = _file_itm_data.pages_info; - auto& page_nesting = _file_itm_data.page_nesting_info; - auto& page_nesting_decode = _file_itm_data.page_nesting_decode_info; + auto& chunks = _file_itm_data.chunks; + auto& pages = _file_itm_data.pages_info; + auto& page_nesting = _file_itm_data.page_nesting_info; + auto& page_nesting_decode = _file_itm_data.page_nesting_decode_info; + auto const level_type_size = _file_itm_data.level_type_size; + + // temporary space for DELTA_BYTE_ARRAY decoding. this only needs to live until + // gpu::DecodeDeltaByteArray returns. + rmm::device_buffer delta_temp_buf; // Should not reach here if there is no page data. CUDF_EXPECTS(pages.size() > 0, "There is no page to decode"); @@ -51,11 +55,12 @@ void reader::impl::decode_page_data(size_t skip_rows, size_t num_rows) // doing a gather operation later on. // TODO: This step is somewhat redundant if size info has already been calculated (nested schema, // chunked reader). - auto const has_strings = (kernel_mask & gpu::KERNEL_MASK_STRING) != 0; + auto const has_strings = + (kernel_mask & (gpu::KERNEL_MASK_STRING | gpu::KERNEL_MASK_DELTA_BYTE_ARRAY)) != 0; std::vector col_sizes(_input_columns.size(), 0L); if (has_strings) { gpu::ComputePageStringSizes( - pages, chunks, skip_rows, num_rows, _file_itm_data.level_type_size, _stream); + pages, chunks, delta_temp_buf, skip_rows, num_rows, level_type_size, kernel_mask, _stream); col_sizes = calculate_page_string_offsets(); @@ -167,8 +172,6 @@ void reader::impl::decode_page_data(size_t skip_rows, size_t num_rows) int const nkernels = std::bitset<32>(kernel_mask).count(); auto streams = cudf::detail::fork_streams(_stream, nkernels); - auto const level_type_size = _file_itm_data.level_type_size; - // launch string decoder int s_idx = 0; if (has_strings) { @@ -177,6 +180,12 @@ void reader::impl::decode_page_data(size_t skip_rows, size_t num_rows) gpu::DecodeStringPageData(pages, chunks, num_rows, skip_rows, level_type_size, stream); } + // launch delta byte array decoder + if ((kernel_mask & gpu::KERNEL_MASK_DELTA_BYTE_ARRAY) != 0) { + gpu::DecodeDeltaByteArray( + pages, chunks, num_rows, skip_rows, level_type_size, streams[s_idx++]); + } + // launch delta binary decoder if ((kernel_mask & gpu::KERNEL_MASK_DELTA_BINARY) != 0) { gpu::DecodeDeltaBinary(pages, chunks, num_rows, skip_rows, level_type_size, streams[s_idx++]); diff --git a/cpp/src/io/parquet/reader_impl_preprocess.cu b/cpp/src/io/parquet/reader_impl_preprocess.cu index bde73c3dd96..483c0f59bb6 100644 --- a/cpp/src/io/parquet/reader_impl_preprocess.cu +++ b/cpp/src/io/parquet/reader_impl_preprocess.cu @@ -324,6 +324,7 @@ constexpr bool is_supported_encoding(Encoding enc) case Encoding::PLAIN_DICTIONARY: case Encoding::RLE: case Encoding::RLE_DICTIONARY: + case Encoding::DELTA_BYTE_ARRAY: case Encoding::DELTA_BINARY_PACKED: return true; default: return false; } From d4a5ae5c5e6e06dbff18da5741495abd21810749 Mon Sep 17 00:00:00 2001 From: seidl Date: Wed, 13 Sep 2023 16:30:07 -0700 Subject: [PATCH 02/39] switch to scoped enum --- cpp/src/io/parquet/page_data.cu | 11 ++++-- cpp/src/io/parquet/page_decode.cuh | 11 ++++-- cpp/src/io/parquet/page_delta_decode.cu | 4 +-- cpp/src/io/parquet/page_hdr.cu | 16 ++++----- cpp/src/io/parquet/page_string_decode.cu | 39 +++++++++++--------- cpp/src/io/parquet/parquet_gpu.hpp | 46 ++++++++++++++++++++---- cpp/src/io/parquet/reader_impl.cpp | 12 +++---- 7 files changed, 96 insertions(+), 43 deletions(-) diff --git a/cpp/src/io/parquet/page_data.cu b/cpp/src/io/parquet/page_data.cu index c26802aa3c2..a0e48678536 100644 --- a/cpp/src/io/parquet/page_data.cu +++ b/cpp/src/io/parquet/page_data.cu @@ -447,8 +447,13 @@ __global__ void __launch_bounds__(decode_block_size) gpuDecodePageData( int out_thread0; [[maybe_unused]] null_count_back_copier _{s, t}; - if (!setupLocalPageInfo( - s, &pages[page_idx], chunks, min_row, num_rows, mask_filter{KERNEL_MASK_GENERAL}, true)) { + if (!setupLocalPageInfo(s, + &pages[page_idx], + chunks, + min_row, + num_rows, + mask_filter{DecodeKernelMask::GENERAL}, + true)) { return; } @@ -599,7 +604,7 @@ __global__ void __launch_bounds__(decode_block_size) gpuDecodePageData( } struct mask_tform { - __device__ uint32_t operator()(PageInfo const& p) { return p.kernel_mask; } + __device__ uint32_t operator()(PageInfo const& p) { return static_cast(p.kernel_mask); } }; } // anonymous namespace diff --git a/cpp/src/io/parquet/page_decode.cuh b/cpp/src/io/parquet/page_decode.cuh index e780256956f..89b44e575ee 100644 --- a/cpp/src/io/parquet/page_decode.cuh +++ b/cpp/src/io/parquet/page_decode.cuh @@ -970,8 +970,15 @@ struct all_types_filter { * @brief Functor for setupLocalPageInfo that takes a mask of allowed types. */ struct mask_filter { - int mask; - __device__ inline bool operator()(PageInfo const& page) { return (page.kernel_mask & mask) != 0; } + uint32_t mask; + + __device__ mask_filter(uint32_t m) : mask(m) {} + __device__ mask_filter(DecodeKernelMask m) : mask(static_cast(m)) {} + + __device__ inline bool operator()(PageInfo const& page) + { + return BitAnd(mask, page.kernel_mask) != 0; + } }; /** diff --git a/cpp/src/io/parquet/page_delta_decode.cu b/cpp/src/io/parquet/page_delta_decode.cu index a9e4cceade0..f26314a0812 100644 --- a/cpp/src/io/parquet/page_delta_decode.cu +++ b/cpp/src/io/parquet/page_delta_decode.cu @@ -322,7 +322,7 @@ __global__ void __launch_bounds__(96) gpuDecodeDeltaBinary( chunks, min_row, num_rows, - mask_filter{KERNEL_MASK_DELTA_BINARY}, + mask_filter{DecodeKernelMask::DELTA_BINARY}, true)) { return; } @@ -447,7 +447,7 @@ __global__ void __launch_bounds__(decode_block_size) gpuDecodeDeltaByteArray( chunks, min_row, num_rows, - mask_filter{KERNEL_MASK_DELTA_BYTE_ARRAY}, + mask_filter{DecodeKernelMask::DELTA_BYTE_ARRAY}, true)) { return; } diff --git a/cpp/src/io/parquet/page_hdr.cu b/cpp/src/io/parquet/page_hdr.cu index 921987b9d40..d2d2f21c796 100644 --- a/cpp/src/io/parquet/page_hdr.cu +++ b/cpp/src/io/parquet/page_hdr.cu @@ -161,21 +161,21 @@ __device__ void skip_struct_field(byte_stream_s* bs, int field_type) * @param chunk Column chunk the page belongs to * @return `kernel_mask_bits` value for the given page */ -__device__ uint32_t kernel_mask_for_page(gpu::PageInfo const& page, - gpu::ColumnChunkDesc const& chunk) +__device__ DecodeKernelMask kernel_mask_for_page(gpu::PageInfo const& page, + gpu::ColumnChunkDesc const& chunk) { - if (page.flags & PAGEINFO_FLAGS_DICTIONARY) { return 0; } + if (page.flags & PAGEINFO_FLAGS_DICTIONARY) { return DecodeKernelMask::NONE; } if (page.encoding == Encoding::DELTA_BINARY_PACKED) { - return KERNEL_MASK_DELTA_BINARY; + return DecodeKernelMask::DELTA_BINARY; } else if (page.encoding == Encoding::DELTA_BYTE_ARRAY) { - return KERNEL_MASK_DELTA_BYTE_ARRAY; + return DecodeKernelMask::DELTA_BYTE_ARRAY; } else if (is_string_col(chunk)) { - return KERNEL_MASK_STRING; + return DecodeKernelMask::STRING; } // non-string, non-delta - return KERNEL_MASK_GENERAL; + return DecodeKernelMask::GENERAL; } /** @@ -396,7 +396,7 @@ __global__ void __launch_bounds__(128) bs->page.str_bytes = 0; bs->page.temp_string_size = 0; bs->page.temp_string_buf = nullptr; - bs->page.kernel_mask = 0; + bs->page.kernel_mask = DecodeKernelMask::NONE; } num_values = bs->ck.num_values; page_info = bs->ck.page_info; diff --git a/cpp/src/io/parquet/page_string_decode.cu b/cpp/src/io/parquet/page_string_decode.cu index 1e1ae0a38f7..e0692d92745 100644 --- a/cpp/src/io/parquet/page_string_decode.cu +++ b/cpp/src/io/parquet/page_string_decode.cu @@ -594,13 +594,14 @@ __global__ void __launch_bounds__(preprocess_block_size) gpuComputeStringPageBou rle_stream decoders[level_type::NUM_LEVEL_TYPES] = {{def_runs}, {rep_runs}}; // setup page info - if (!setupLocalPageInfo(s, - pp, - chunks, - min_row, - num_rows, - mask_filter{KERNEL_MASK_STRING | KERNEL_MASK_DELTA_BYTE_ARRAY}, - true)) { + if (!setupLocalPageInfo( + s, + pp, + chunks, + min_row, + num_rows, + mask_filter{BitOr(DecodeKernelMask::STRING, DecodeKernelMask::DELTA_BYTE_ARRAY)}, + true)) { return; } @@ -647,7 +648,7 @@ __global__ void __launch_bounds__(delta_preproc_block_size) gpuComputeDeltaPageS // setup page info if (!setupLocalPageInfo( - s, pp, chunks, min_row, num_rows, mask_filter{KERNEL_MASK_DELTA_BYTE_ARRAY}, true)) { + s, pp, chunks, min_row, num_rows, mask_filter{DecodeKernelMask::DELTA_BYTE_ARRAY}, true)) { return; } @@ -714,7 +715,7 @@ __global__ void __launch_bounds__(preprocess_block_size) gpuComputePageStringSiz // setup page info if (!setupLocalPageInfo( - s, pp, chunks, min_row, num_rows, mask_filter{KERNEL_MASK_STRING}, true)) { + s, pp, chunks, min_row, num_rows, mask_filter{DecodeKernelMask::STRING}, true)) { return; } @@ -803,8 +804,13 @@ __global__ void __launch_bounds__(decode_block_size) gpuDecodeStringPageData( int const lane_id = t % warp_size; [[maybe_unused]] null_count_back_copier _{s, t}; - if (!setupLocalPageInfo( - s, &pages[page_idx], chunks, min_row, num_rows, mask_filter{KERNEL_MASK_STRING}, true)) { + if (!setupLocalPageInfo(s, + &pages[page_idx], + chunks, + min_row, + num_rows, + mask_filter{DecodeKernelMask::STRING}, + true)) { return; } @@ -985,17 +991,18 @@ void ComputePageStringSizes(cudf::detail::hostdevice_vector& pages, } // kernel mask may contain other kernels we don't need to count - int count_mask = kernel_mask & (KERNEL_MASK_DELTA_BYTE_ARRAY | KERNEL_MASK_STRING); - int nkernels = std::bitset<32>(count_mask).count(); - auto streams = cudf::detail::fork_streams(stream, nkernels); + int count_mask = + kernel_mask & BitOr(DecodeKernelMask::DELTA_BYTE_ARRAY, DecodeKernelMask::STRING); + int nkernels = std::bitset<32>(count_mask).count(); + auto streams = cudf::detail::fork_streams(stream, nkernels); int s_idx = 0; - if ((kernel_mask & KERNEL_MASK_DELTA_BYTE_ARRAY) != 0) { + if (BitAnd(kernel_mask, DecodeKernelMask::DELTA_BYTE_ARRAY) != 0) { dim3 dim_delta(delta_preproc_block_size, 1); gpuComputeDeltaPageStringSizes<<>>( pages.device_ptr(), chunks, min_row, num_rows); } - if ((kernel_mask & KERNEL_MASK_STRING) != 0) { + if (BitAnd(kernel_mask, DecodeKernelMask::STRING) != 0) { gpuComputePageStringSizes<<>>( pages.device_ptr(), chunks, min_row, num_rows); } diff --git a/cpp/src/io/parquet/parquet_gpu.hpp b/cpp/src/io/parquet/parquet_gpu.hpp index d6a872ac3a8..cf8a956b9de 100644 --- a/cpp/src/io/parquet/parquet_gpu.hpp +++ b/cpp/src/io/parquet/parquet_gpu.hpp @@ -34,6 +34,7 @@ #include +#include #include namespace cudf::io::parquet { @@ -76,6 +77,38 @@ struct input_column_info { namespace gpu { +// TODO this is in C++23 +template +struct is_scoped_enum { + static const bool value = + std::is_enum_v and not std::is_convertible_v>; +}; + +// helpers to do bit operations on scoped enums +template ::value, bool> = true> +constexpr uint32_t BitAnd(Enum a, Enum b) +{ + return static_cast(a) & static_cast(b); +} + +template ::value, bool> = true> +constexpr uint32_t BitAnd(uint32_t a, Enum b) +{ + return a & static_cast(b); +} + +template ::value, bool> = true> +constexpr uint32_t BitOr(Enum a, Enum b) +{ + return static_cast(a) | static_cast(b); +} + +template ::value, bool> = true> +constexpr uint32_t BitOr(uint32_t a, Enum b) +{ + return a | static_cast(b); +} + /** * @brief Enums for the flags in the page header */ @@ -99,11 +132,12 @@ enum level_type { * * Used to control which decode kernels to run. */ -enum kernel_mask_bits { - KERNEL_MASK_GENERAL = (1 << 0), // Run catch-all decode kernel - KERNEL_MASK_STRING = (1 << 1), // Run decode kernel for string data - KERNEL_MASK_DELTA_BINARY = (1 << 2), // Run decode kernel for DELTA_BINARY_PACKED data - KERNEL_MASK_DELTA_BYTE_ARRAY = (1 << 3) // Run decode kernel for DELTA_BYTE_ARRAY encoded data +enum class DecodeKernelMask { + NONE = 0, + GENERAL = (1 << 0), // Run catch-all decode kernel + STRING = (1 << 1), // Run decode kernel for string data + DELTA_BINARY = (1 << 2), // Run decode kernel for DELTA_BINARY_PACKED data + DELTA_BYTE_ARRAY = (1 << 3) // Run decode kernel for DELTA_BYTE_ARRAY encoded data }; /** @@ -229,7 +263,7 @@ struct PageInfo { int64_t temp_string_size; uint8_t* temp_string_buf; - uint32_t kernel_mask; + DecodeKernelMask kernel_mask; }; /** diff --git a/cpp/src/io/parquet/reader_impl.cpp b/cpp/src/io/parquet/reader_impl.cpp index f062ce9832d..972f7efb4b9 100644 --- a/cpp/src/io/parquet/reader_impl.cpp +++ b/cpp/src/io/parquet/reader_impl.cpp @@ -55,8 +55,8 @@ void reader::impl::decode_page_data(size_t skip_rows, size_t num_rows) // doing a gather operation later on. // TODO: This step is somewhat redundant if size info has already been calculated (nested schema, // chunked reader). - auto const has_strings = - (kernel_mask & (gpu::KERNEL_MASK_STRING | gpu::KERNEL_MASK_DELTA_BYTE_ARRAY)) != 0; + auto const has_strings = (kernel_mask & BitOr(gpu::DecodeKernelMask::STRING, + gpu::DecodeKernelMask::DELTA_BYTE_ARRAY)) != 0; std::vector col_sizes(_input_columns.size(), 0L); if (has_strings) { gpu::ComputePageStringSizes( @@ -174,25 +174,25 @@ void reader::impl::decode_page_data(size_t skip_rows, size_t num_rows) // launch string decoder int s_idx = 0; - if (has_strings) { + if (BitAnd(kernel_mask, gpu::DecodeKernelMask::STRING) != 0) { auto& stream = streams[s_idx++]; chunk_nested_str_data.host_to_device_async(stream); gpu::DecodeStringPageData(pages, chunks, num_rows, skip_rows, level_type_size, stream); } // launch delta byte array decoder - if ((kernel_mask & gpu::KERNEL_MASK_DELTA_BYTE_ARRAY) != 0) { + if (BitAnd(kernel_mask, gpu::DecodeKernelMask::DELTA_BYTE_ARRAY) != 0) { gpu::DecodeDeltaByteArray( pages, chunks, num_rows, skip_rows, level_type_size, streams[s_idx++]); } // launch delta binary decoder - if ((kernel_mask & gpu::KERNEL_MASK_DELTA_BINARY) != 0) { + if (BitAnd(kernel_mask, gpu::DecodeKernelMask::DELTA_BINARY) != 0) { gpu::DecodeDeltaBinary(pages, chunks, num_rows, skip_rows, level_type_size, streams[s_idx++]); } // launch the catch-all page decoder - if ((kernel_mask & gpu::KERNEL_MASK_GENERAL) != 0) { + if (BitAnd(kernel_mask, gpu::DecodeKernelMask::GENERAL) != 0) { gpu::DecodePageData(pages, chunks, num_rows, skip_rows, level_type_size, streams[s_idx++]); } From 25a42eb76cc2f0efdf2203cb6e47a1d3def23de6 Mon Sep 17 00:00:00 2001 From: seidl Date: Wed, 20 Sep 2023 10:44:49 -0700 Subject: [PATCH 03/39] need to copy string data on default stream --- cpp/src/io/parquet/reader_impl.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/cpp/src/io/parquet/reader_impl.cpp b/cpp/src/io/parquet/reader_impl.cpp index 972f7efb4b9..c6d3a2dee81 100644 --- a/cpp/src/io/parquet/reader_impl.cpp +++ b/cpp/src/io/parquet/reader_impl.cpp @@ -167,6 +167,7 @@ void reader::impl::decode_page_data(size_t skip_rows, size_t num_rows) chunks.host_to_device_async(_stream); chunk_nested_valids.host_to_device_async(_stream); chunk_nested_data.host_to_device_async(_stream); + if (has_strings) { chunk_nested_str_data.host_to_device_async(_stream); } // get the number of streams we need from the pool and tell them to wait on the H2D copies int const nkernels = std::bitset<32>(kernel_mask).count(); @@ -175,9 +176,8 @@ void reader::impl::decode_page_data(size_t skip_rows, size_t num_rows) // launch string decoder int s_idx = 0; if (BitAnd(kernel_mask, gpu::DecodeKernelMask::STRING) != 0) { - auto& stream = streams[s_idx++]; - chunk_nested_str_data.host_to_device_async(stream); - gpu::DecodeStringPageData(pages, chunks, num_rows, skip_rows, level_type_size, stream); + gpu::DecodeStringPageData( + pages, chunks, num_rows, skip_rows, level_type_size, streams[s_idx++]); } // launch delta byte array decoder From 9b2a67e42684ee29ea87ab7d986aa51437d828bd Mon Sep 17 00:00:00 2001 From: seidl Date: Thu, 28 Sep 2023 13:15:34 -0700 Subject: [PATCH 04/39] set error code in delta byte array decoder --- cpp/src/io/parquet/page_delta_decode.cu | 21 +++++++++++++++------ 1 file changed, 15 insertions(+), 6 deletions(-) diff --git a/cpp/src/io/parquet/page_delta_decode.cu b/cpp/src/io/parquet/page_delta_decode.cu index 05434885a7c..6ee98e79adf 100644 --- a/cpp/src/io/parquet/page_delta_decode.cu +++ b/cpp/src/io/parquet/page_delta_decode.cu @@ -434,8 +434,12 @@ __global__ void __launch_bounds__(96) // suffixes are not encoded in the header, we're going to have to first do a quick pass through them // to find the start/end of each structure. template -__global__ void __launch_bounds__(decode_block_size) gpuDecodeDeltaByteArray( - PageInfo* pages, device_span chunks, size_t min_row, size_t num_rows) +__global__ void __launch_bounds__(decode_block_size) + gpuDecodeDeltaByteArray(PageInfo* pages, + device_span chunks, + size_t min_row, + size_t num_rows, + int32_t* error_code) { using cudf::detail::warp_size; __shared__ __align__(16) delta_byte_array_decoder db_state; @@ -567,6 +571,11 @@ __global__ void __launch_bounds__(decode_block_size) gpuDecodeDeltaByteArray( auto const offptr = reinterpret_cast(nesting_info_base[leaf_level_index].data_out); block_excl_sum(offptr, value_count, s->page.str_offset); + + if (t == 0 and s->error != 0) { + cuda::atomic_ref ref{*error_code}; + ref.fetch_or(s->error, cuda::std::memory_order_relaxed); + } } } // anonymous namespace @@ -613,11 +622,11 @@ void __host__ DecodeDeltaByteArray(cudf::detail::hostdevice_vector& pa dim3 dim_grid(pages.size(), 1); // 1 threadblock per page if (level_type_size == 1) { - gpuDecodeDeltaByteArray - <<>>(pages.device_ptr(), chunks, min_row, num_rows); + gpuDecodeDeltaByteArray<<>>( + pages.device_ptr(), chunks, min_row, num_rows, error_code); } else { - gpuDecodeDeltaByteArray - <<>>(pages.device_ptr(), chunks, min_row, num_rows); + gpuDecodeDeltaByteArray<<>>( + pages.device_ptr(), chunks, min_row, num_rows, error_code); } } From 20dc29f4edfbb1785eed2d25ae516bdc16543a1d Mon Sep 17 00:00:00 2001 From: seidl Date: Fri, 29 Sep 2023 14:57:20 -0700 Subject: [PATCH 05/39] redo BitAnd and BitOr --- cpp/src/io/parquet/parquet_gpu.hpp | 47 +++++++++++++++++------------- 1 file changed, 27 insertions(+), 20 deletions(-) diff --git a/cpp/src/io/parquet/parquet_gpu.hpp b/cpp/src/io/parquet/parquet_gpu.hpp index 20450390634..9a31ddb6e6c 100644 --- a/cpp/src/io/parquet/parquet_gpu.hpp +++ b/cpp/src/io/parquet/parquet_gpu.hpp @@ -92,38 +92,45 @@ struct input_column_info { namespace gpu { -// TODO this is in C++23 -template +// this is in C++23 +#if !defined(__cpp_lib_is_scoped_enum) +template > struct is_scoped_enum { - static const bool value = - std::is_enum_v and not std::is_convertible_v>; + static const bool value = not std::is_convertible_v>; +}; + +template +struct is_scoped_enum { + static const bool value = false; }; +#else +using std::is_scoped_enum; +#endif // helpers to do bit operations on scoped enums -template ::value, bool> = true> -constexpr uint32_t BitAnd(Enum a, Enum b) +template ::value and std::is_same_v) or + (is_scoped_enum::value and std::is_same_v) or + (is_scoped_enum::value and std::is_same_v)>* = + nullptr> +constexpr uint32_t BitAnd(T1 a, T2 b) { return static_cast(a) & static_cast(b); } -template ::value, bool> = true> -constexpr uint32_t BitAnd(uint32_t a, Enum b) -{ - return a & static_cast(b); -} - -template ::value, bool> = true> -constexpr uint32_t BitOr(Enum a, Enum b) +// helpers to do bit operations on scoped enums +template ::value and std::is_same_v) or + (is_scoped_enum::value and std::is_same_v) or + (is_scoped_enum::value and std::is_same_v)>* = + nullptr> +constexpr uint32_t BitOr(T1 a, T2 b) { return static_cast(a) | static_cast(b); } -template ::value, bool> = true> -constexpr uint32_t BitOr(uint32_t a, Enum b) -{ - return a | static_cast(b); -} - /** * @brief Enums for the flags in the page header */ From cba285820f437d9f8ca0a1ee3a9a88bcae52fe9c Mon Sep 17 00:00:00 2001 From: seidl Date: Fri, 29 Sep 2023 15:02:35 -0700 Subject: [PATCH 06/39] use snake case for names --- cpp/src/io/parquet/page_data.cu | 2 +- cpp/src/io/parquet/page_decode.cuh | 2 +- cpp/src/io/parquet/page_delta_decode.cu | 4 ++-- cpp/src/io/parquet/page_hdr.cu | 16 ++++++++-------- cpp/src/io/parquet/page_string_decode.cu | 21 +++++++++++++-------- cpp/src/io/parquet/parquet_gpu.hpp | 4 ++-- cpp/src/io/parquet/reader_impl.cpp | 12 ++++++------ 7 files changed, 33 insertions(+), 28 deletions(-) diff --git a/cpp/src/io/parquet/page_data.cu b/cpp/src/io/parquet/page_data.cu index 027dc7f6407..9414c6c71cf 100644 --- a/cpp/src/io/parquet/page_data.cu +++ b/cpp/src/io/parquet/page_data.cu @@ -457,7 +457,7 @@ __global__ void __launch_bounds__(decode_block_size) chunks, min_row, num_rows, - mask_filter{DecodeKernelMask::GENERAL}, + mask_filter{decode_kernel_mask::GENERAL}, true)) { return; } diff --git a/cpp/src/io/parquet/page_decode.cuh b/cpp/src/io/parquet/page_decode.cuh index 259abe1c63b..e2f122d327b 100644 --- a/cpp/src/io/parquet/page_decode.cuh +++ b/cpp/src/io/parquet/page_decode.cuh @@ -994,7 +994,7 @@ struct mask_filter { uint32_t mask; __device__ mask_filter(uint32_t m) : mask(m) {} - __device__ mask_filter(DecodeKernelMask m) : mask(static_cast(m)) {} + __device__ mask_filter(decode_kernel_mask m) : mask(static_cast(m)) {} __device__ inline bool operator()(PageInfo const& page) { diff --git a/cpp/src/io/parquet/page_delta_decode.cu b/cpp/src/io/parquet/page_delta_decode.cu index 6ee98e79adf..6fc8aa8f722 100644 --- a/cpp/src/io/parquet/page_delta_decode.cu +++ b/cpp/src/io/parquet/page_delta_decode.cu @@ -326,7 +326,7 @@ __global__ void __launch_bounds__(96) chunks, min_row, num_rows, - mask_filter{DecodeKernelMask::DELTA_BINARY}, + mask_filter{decode_kernel_mask::DELTA_BINARY}, true)) { return; } @@ -461,7 +461,7 @@ __global__ void __launch_bounds__(decode_block_size) chunks, min_row, num_rows, - mask_filter{DecodeKernelMask::DELTA_BYTE_ARRAY}, + mask_filter{decode_kernel_mask::DELTA_BYTE_ARRAY}, true)) { return; } diff --git a/cpp/src/io/parquet/page_hdr.cu b/cpp/src/io/parquet/page_hdr.cu index 5aedd4a7fbf..9c5eb8d541d 100644 --- a/cpp/src/io/parquet/page_hdr.cu +++ b/cpp/src/io/parquet/page_hdr.cu @@ -161,21 +161,21 @@ __device__ void skip_struct_field(byte_stream_s* bs, int field_type) * @param chunk Column chunk the page belongs to * @return `kernel_mask_bits` value for the given page */ -__device__ DecodeKernelMask kernel_mask_for_page(gpu::PageInfo const& page, - gpu::ColumnChunkDesc const& chunk) +__device__ decode_kernel_mask kernel_mask_for_page(gpu::PageInfo const& page, + gpu::ColumnChunkDesc const& chunk) { - if (page.flags & PAGEINFO_FLAGS_DICTIONARY) { return DecodeKernelMask::NONE; } + if (page.flags & PAGEINFO_FLAGS_DICTIONARY) { return decode_kernel_mask::NONE; } if (page.encoding == Encoding::DELTA_BINARY_PACKED) { - return DecodeKernelMask::DELTA_BINARY; + return decode_kernel_mask::DELTA_BINARY; } else if (page.encoding == Encoding::DELTA_BYTE_ARRAY) { - return DecodeKernelMask::DELTA_BYTE_ARRAY; + return decode_kernel_mask::DELTA_BYTE_ARRAY; } else if (is_string_col(chunk)) { - return DecodeKernelMask::STRING; + return decode_kernel_mask::STRING; } // non-string, non-delta - return DecodeKernelMask::GENERAL; + return decode_kernel_mask::GENERAL; } /** @@ -396,7 +396,7 @@ __global__ void __launch_bounds__(128) bs->page.str_bytes = 0; bs->page.temp_string_size = 0; bs->page.temp_string_buf = nullptr; - bs->page.kernel_mask = DecodeKernelMask::NONE; + bs->page.kernel_mask = decode_kernel_mask::NONE; } num_values = bs->ck.num_values; page_info = bs->ck.page_info; diff --git a/cpp/src/io/parquet/page_string_decode.cu b/cpp/src/io/parquet/page_string_decode.cu index e26aee7c295..9a036d93734 100644 --- a/cpp/src/io/parquet/page_string_decode.cu +++ b/cpp/src/io/parquet/page_string_decode.cu @@ -600,7 +600,7 @@ __global__ void __launch_bounds__(preprocess_block_size) gpuComputeStringPageBou chunks, min_row, num_rows, - mask_filter{BitOr(DecodeKernelMask::STRING, DecodeKernelMask::DELTA_BYTE_ARRAY)}, + mask_filter{BitOr(decode_kernel_mask::STRING, decode_kernel_mask::DELTA_BYTE_ARRAY)}, true)) { return; } @@ -647,8 +647,13 @@ __global__ void __launch_bounds__(delta_preproc_block_size) gpuComputeDeltaPageS bool const has_repetition = chunks[pp->chunk_idx].max_level[level_type::REPETITION] > 0; // setup page info - if (!setupLocalPageInfo( - s, pp, chunks, min_row, num_rows, mask_filter{DecodeKernelMask::DELTA_BYTE_ARRAY}, true)) { + if (!setupLocalPageInfo(s, + pp, + chunks, + min_row, + num_rows, + mask_filter{decode_kernel_mask::DELTA_BYTE_ARRAY}, + true)) { return; } @@ -715,7 +720,7 @@ __global__ void __launch_bounds__(preprocess_block_size) gpuComputePageStringSiz // setup page info if (!setupLocalPageInfo( - s, pp, chunks, min_row, num_rows, mask_filter{DecodeKernelMask::STRING}, true)) { + s, pp, chunks, min_row, num_rows, mask_filter{decode_kernel_mask::STRING}, true)) { return; } @@ -813,7 +818,7 @@ __global__ void __launch_bounds__(decode_block_size) chunks, min_row, num_rows, - mask_filter{DecodeKernelMask::STRING}, + mask_filter{decode_kernel_mask::STRING}, true)) { return; } @@ -1002,17 +1007,17 @@ void ComputePageStringSizes(cudf::detail::hostdevice_vector& pages, // kernel mask may contain other kernels we don't need to count int count_mask = - kernel_mask & BitOr(DecodeKernelMask::DELTA_BYTE_ARRAY, DecodeKernelMask::STRING); + kernel_mask & BitOr(decode_kernel_mask::DELTA_BYTE_ARRAY, decode_kernel_mask::STRING); int nkernels = std::bitset<32>(count_mask).count(); auto streams = cudf::detail::fork_streams(stream, nkernels); int s_idx = 0; - if (BitAnd(kernel_mask, DecodeKernelMask::DELTA_BYTE_ARRAY) != 0) { + if (BitAnd(kernel_mask, decode_kernel_mask::DELTA_BYTE_ARRAY) != 0) { dim3 dim_delta(delta_preproc_block_size, 1); gpuComputeDeltaPageStringSizes<<>>( pages.device_ptr(), chunks, min_row, num_rows); } - if (BitAnd(kernel_mask, DecodeKernelMask::STRING) != 0) { + if (BitAnd(kernel_mask, decode_kernel_mask::STRING) != 0) { gpuComputePageStringSizes<<>>( pages.device_ptr(), chunks, min_row, num_rows); } diff --git a/cpp/src/io/parquet/parquet_gpu.hpp b/cpp/src/io/parquet/parquet_gpu.hpp index 9a31ddb6e6c..12a5daecbd8 100644 --- a/cpp/src/io/parquet/parquet_gpu.hpp +++ b/cpp/src/io/parquet/parquet_gpu.hpp @@ -154,7 +154,7 @@ enum level_type { * * Used to control which decode kernels to run. */ -enum class DecodeKernelMask { +enum class decode_kernel_mask { NONE = 0, GENERAL = (1 << 0), // Run catch-all decode kernel STRING = (1 << 1), // Run decode kernel for string data @@ -285,7 +285,7 @@ struct PageInfo { int64_t temp_string_size; uint8_t* temp_string_buf; - DecodeKernelMask kernel_mask; + decode_kernel_mask kernel_mask; }; /** diff --git a/cpp/src/io/parquet/reader_impl.cpp b/cpp/src/io/parquet/reader_impl.cpp index f1abab0c23a..18fa1e4f155 100644 --- a/cpp/src/io/parquet/reader_impl.cpp +++ b/cpp/src/io/parquet/reader_impl.cpp @@ -55,8 +55,8 @@ void reader::impl::decode_page_data(size_t skip_rows, size_t num_rows) // doing a gather operation later on. // TODO: This step is somewhat redundant if size info has already been calculated (nested schema, // chunked reader). - auto const has_strings = (kernel_mask & BitOr(gpu::DecodeKernelMask::STRING, - gpu::DecodeKernelMask::DELTA_BYTE_ARRAY)) != 0; + auto const has_strings = (kernel_mask & BitOr(gpu::decode_kernel_mask::STRING, + gpu::decode_kernel_mask::DELTA_BYTE_ARRAY)) != 0; std::vector col_sizes(_input_columns.size(), 0L); if (has_strings) { gpu::ComputePageStringSizes( @@ -177,25 +177,25 @@ void reader::impl::decode_page_data(size_t skip_rows, size_t num_rows) // launch string decoder int s_idx = 0; - if (BitAnd(kernel_mask, gpu::DecodeKernelMask::STRING) != 0) { + if (BitAnd(kernel_mask, gpu::decode_kernel_mask::STRING) != 0) { gpu::DecodeStringPageData( pages, chunks, num_rows, skip_rows, level_type_size, error_code.data(), streams[s_idx++]); } // launch delta byte array decoder - if (BitAnd(kernel_mask, gpu::DecodeKernelMask::DELTA_BYTE_ARRAY) != 0) { + if (BitAnd(kernel_mask, gpu::decode_kernel_mask::DELTA_BYTE_ARRAY) != 0) { gpu::DecodeDeltaByteArray( pages, chunks, num_rows, skip_rows, level_type_size, error_code.data(), streams[s_idx++]); } // launch delta binary decoder - if (BitAnd(kernel_mask, gpu::DecodeKernelMask::DELTA_BINARY) != 0) { + if (BitAnd(kernel_mask, gpu::decode_kernel_mask::DELTA_BINARY) != 0) { gpu::DecodeDeltaBinary( pages, chunks, num_rows, skip_rows, level_type_size, error_code.data(), streams[s_idx++]); } // launch the catch-all page decoder - if (BitAnd(kernel_mask, gpu::DecodeKernelMask::GENERAL) != 0) { + if (BitAnd(kernel_mask, gpu::decode_kernel_mask::GENERAL) != 0) { gpu::DecodePageData( pages, chunks, num_rows, skip_rows, level_type_size, error_code.data(), streams[s_idx++]); } From c52481e67df908ac9565fe10d4c266c3f91ce0e3 Mon Sep 17 00:00:00 2001 From: seidl Date: Fri, 29 Sep 2023 15:27:38 -0700 Subject: [PATCH 07/39] add test --- .../tests/data/parquet/delta_byte_arr.parquet | Bin 0 -> 5783 bytes python/cudf/cudf/tests/test_parquet.py | 5 +++++ 2 files changed, 5 insertions(+) create mode 100644 python/cudf/cudf/tests/data/parquet/delta_byte_arr.parquet diff --git a/python/cudf/cudf/tests/data/parquet/delta_byte_arr.parquet b/python/cudf/cudf/tests/data/parquet/delta_byte_arr.parquet new file mode 100644 index 0000000000000000000000000000000000000000..7f6006a75bff0498e373d61f1265ff2e8458c917 GIT binary patch literal 5783 zcmcII2V4``(;E^(DAHut1UN+xLQyH9NIw=L9WDW-1Uw8R0VI)_glYp74;un177(#e z&J(bKD5wY)5JcrpKtWMdEC^Ttm9zYJ7lNGU-TlAc_kEjRcIVB!nKy4{cHYb~U_UdE z2-e7fXJkS_4hSHBWsm>@YT7}n@c6~>rDG6z0t5SuVE_Q&;0!8EJ_7DL2*80fb*P}w zw)j)`;!Xt*u6&`|i}u2Fc{z85&!0o*d-R73?#eZWXB;>bf2^`g3SGt^N3An!Yfq@j z)~58>9NjXK(j#lBbH6?KH8)2uj2EobJjd%?^YKSb*rQ=*aLx)j9!43TfIaZ$P1zo)Qrj5hs?!3b>-MXbz|ZaiA&^REejO?EkxMA~wp$ zgH@P~2$+qOP#akh4DkNZA~K#XGDd8HKhMD7Wyb6wAONj$+PaxG-(C6Rw}fN2Z4%2R zAC|At>mA)(xoyga(l?J7-ZDlcH3d3xb>*Iqx4V|K-9v!kgzoCkZ{ozhmSW$PgS~kJ zS@!bIu1=BL&i2Oqjs-XcfX9Ei zZiOfJ{t4x(X$y!3%YzyPnRh8d8L26hBErOMew>6YiH=>0S>ib!T8F3JAlUzTH^`Rj zWWhSZ*I^+S7|rKLvjt>%n9ogQFXeDWlfs~a1W-Yj*^CW)lug;IR6_T<@{^*PK#T00 z$LcdLEiNzMOD7D!8pb`;kZ+Sd8pVr<+)As*Xv- z_{8d+j6CHBaqC>%&rVrRC>iCIQd2wg6dE)u)>F)1rJRbUWnEenW4|c&J}&kxlU_zy zz0>&0ju$U>CSMRYdcDX{t6SkhKP$WWS^9U1(+;Ry5y-b~iJWC#&(D20>eg`P*vVs0 zu2j89pVb<2qI!hfF8*o2knB7suUXhW{fS=O!!paQbMK5aZ=T(&9^6$yl>u-En^V*$ z*`F6J5=+9CvIShs{`o{GUzS>lgyJ_%i}>M;EZBqj;m@Hb8yFQW;gX3kKb$L)pz-(i zOZE@MNy%U83zos1X~ zg;M8}pL~dP#2*&GtGPEW#s2s+qr9v6aq$+t>2#rj6N6zHHHTqYkXlqIcw-vx1ds}x z$PhVVvs0npGV20+g6?$1$Hy3CFKg{xGVCDImL!vXx{W5B`KmJ{Tbmy@osZsHp7rk2 zN8p63=*;)V3*;*5xV)vapB~tEwKDJ6b&u=T@9$4tkrWeGJ-}X-k@)`ntpLZjosPSs zPHwM=+S;#8JL}gfHh9bP{H3!%&*q1-+wA@rQ12b-&=S9o-gaK0`N#_6?9w+OT?xFC zR}{|a`;`h>uh$MD%QzDQ4!#XLczWOB*RGP1ma8KhH)!Di6Tm>~6vc=zK6|oHGKo-? zEcLQ7`c8apX2c;SIB*vZeKG#*5L_@B=lfM2=*IyZd zP#2E+R0CCZTm?kOjU@1nEUF`MQobS@Xasopxq0~Nc=&t6zI6XNItz{cjmdJbw=?YP z0VC_;PkjlZVFe)P4udB2%2xiMr6$~QUlEWey(Hsm6?f%U`V7pSnU(3#hI+g*Ms>U2yUlYy_n_!xO`o=^Q>y=Z0+H&J zt{T1A;D(M2ZY@a;G!0EGxUgo;lqXJ3swAgZwdr|c<&<9P{%SeNTN`EN%`OOvaL4-B?tLHh z+oFxjx5X*i;(u&-x<$})%3lASKfkazPqzBd6Yd%xi)^P$?;7mu-+A5C&yW7NqF!Sq zjeaIZOnJ2Hb%o#1FI5KT*sA9gHafoQfnF4M9Nrge{4B{PZ7{VgY1FVX+4-l8U2RPr zhE`Lkv}tm5gWHuB1*#inj#O~AX1M`v$y(pw;^68gGQ*k(cjn+jvUUw*kQ!P z0&V;zT~iJT`q;?mu|t3k1yHkR*jsS`01?#&f_18>Fd0l32|Md#40Og*zcG^e1?;6x z5pEosh1C+4$2_`(W*iYKAkUgO6O6IsDSl7%_| zYoV^Hom0#$9DXD~pIN^EoR7;@_FYfBI}&i*&co2|YX3f69HBe4&nmQC5cFHku_zA$ zfAOZY2c{~!--i@=5O4!PAE2y0bzfSAVcP7x?Y^6yXPI0ax+Cv0gAxG0t|{x6XOYi^ z7%?4QA2jyV)w5VzaKt7lzV4c672CV7Pt(`x)Wd`OOODMR?%Qyw*mOaKzBJ7JU(=U|FI#`ujp&d*>v%%30_$DN6{B{$ow|35sYN^)cyQaGf#N*1gwkZ| z3ynOL$l(T4rgitJ{?Pi$gQeM3_inS}tkN6k589iiw<^@z+1&a**A_gj?2}fUy(jeZ znWCwzJckz+v|{{r(lqZ3Sp`#0CSA#zwm^6kUb9wajf^HK1@aai-J$NTxaD#RPEpgI z0HoZAve2Zc$M*D(@+z=!V}?i9Yqv z-NEyZ)TWLH1LxiH_?LCdJSu7iJ!i^x{__0VaEt2ufg=Oj>s99G@y`jJ&HQDO0`_XX zI#Z~9e-a( zQX#TRb)OFzq(S!Gh;hOek7hgFT>D(G_eUZ0V7MhdF1jS4+L?b568j7{DCg>BpRP?$ zA(YSS(@ZlfEsr_umeYoRI2A5n5{qUr9qtK6Yv&Gp(we$xu*P!7QQhm#N3^Dztu#Q~ z|Bz{`Pag5|dcb+}_^#Zi>%m=bY#L7AOL~}+zPiz>KSz5(R(t2gIqyPPyKZjxa}d#g ztBLJvmz}%IDlk`k_Khd_eW8-u*%oWXdWwF3({M|kQs0na>*nUr>!TIBViHfhoa<{9 z*xXRaSa#Fp!obD)Z7L_MU6l0Gw1yUJJ(5bz*?BrJsO3DrDEZMX*{2p4tZi+c55#p6 zJA3j6$trfYzW=OGuUm0QvQo$JM(-=5X?r(T8SVuG4qs`vJ9W$0Xi?9NH|<*3UK1^nYrz@zBYn+_bwPT@FW9 zI$Uh*Rmkl3{nZ`%HLRv{aHvjrHN)TGb#M4pp(a;7xve~)INfe^lE~JXHI`p7D(=?+~!x7 zkTavtH?)XrIZp;4C~e2-NmCS(7Lh~~%@1R9I3liCjHSgr)1dY0)Dj(gtN*i%grvMZ zc<{zwS|B-^g(c0(!kUZ$<}@=Tc}W|degbgZo2r*-_^djnvdD9lVtyXku{1H{*NnZNv*TCf?6wVgRm1Avb$cke zTX@f!Wi;PlRzO$%iBpjunRYiKROM(K^%v(XMh3Y zV2KX6UVR+SumDRiksyI`#9fIk62&Hji6f$rw**v2T&^ecRj;cDcxxltV3a?#eqRy3 z^dyrBlk4Ki_-_}ZlV^Ng&`hRD3y;aTuix~^v~NW4I2qzlv?+Pib9ilOwEwR=eJI}n zgrK9=;7V$VvPS^+CHcml~ZK{Is(*B zQzzt1_6R2CpE?!j5h`QE2DAXE7oiCm-}W9R?HjEG^Z^xe*cO}xP*bK)2>IH<82kUE zVuiLlB66E;!36;I>5K`v(%``4O{z+vBaVn)3hYod%4ieQzm6M>?f>C~fa*|f>+HY- zIDlF?Z({v##3rV~KW10x4l3-CJ!lI+n}0yFlr~uj_VfEO(01%yCU?l1pjTp~3-aZS zIsP5a3x*S5mq^$XMIlcZb{?lNkueN=pd$n&eIlb6^v6{9LAj92*_kPo2g5>?o?wllmjuxQB*B4vaS+WlhUMlgucv$()E!|CeKFb*j9thY@*pgo~CS9nl$A1~NE0JVbvvBy%B&S9Yb`}_T~PDc}d`?Cc6q%XYa%5;qW_x_{9lldcPu3}D* zUlhlUA0LkT6b%ArCK@e2!m#yth()4fL{)5p)Sk}HekgM6LIMk!g6-EA#US=s!U^&< z4X2s$BltE#7B Date: Mon, 2 Oct 2023 16:48:08 -0700 Subject: [PATCH 08/39] cleanups --- cpp/src/io/parquet/page_delta_decode.cu | 13 ++++++------- cpp/src/io/parquet/parquet_gpu.hpp | 1 - 2 files changed, 6 insertions(+), 8 deletions(-) diff --git a/cpp/src/io/parquet/page_delta_decode.cu b/cpp/src/io/parquet/page_delta_decode.cu index 6fc8aa8f722..8353fe3d401 100644 --- a/cpp/src/io/parquet/page_delta_decode.cu +++ b/cpp/src/io/parquet/page_delta_decode.cu @@ -40,7 +40,6 @@ struct delta_byte_array_decoder { delta_binary_decoder prefixes; // state of decoder for prefix lengths delta_binary_decoder suffixes; // state of decoder for suffix lengths - // size_type offset[non_zero_buffer_size]; // circular buffer for string output offsets // initialize the prefixes and suffixes blocks __device__ void init(uint8_t const* start, uint8_t const* end, uint32_t start_idx, uint8_t* temp) @@ -137,7 +136,7 @@ struct delta_byte_array_decoder { auto p_temp_out = temp_buf; auto copy_batch = [&](uint8_t* out, uint32_t idx, uint32_t end) { - uint32_t ln_idx = idx + lane_id; + uint32_t const ln_idx = idx + lane_id; // calculate offsets into suffix data uint32_t const src_idx = rolling_index(ln_idx); @@ -145,7 +144,7 @@ struct delta_byte_array_decoder { uint64_t suffix_off = 0; WarpScan(scan_temp).ExclusiveSum(suffix_len, suffix_off); - // calculate offsets into string data and save in string_offsets + // calculate offsets into string data uint64_t const prefix_len = ln_idx < end ? prefixes.value[src_idx] : 0; uint64_t const string_len = prefix_len + suffix_len; @@ -505,7 +504,7 @@ __global__ void __launch_bounds__(decode_block_size) uint32_t target_pos; uint32_t const src_pos = s->src_pos; - if (t < 96) { // warp 0..2 + if (t < 3 * warp_size) { // warp 0..2 target_pos = min(src_pos + 2 * (batch_size), s->nz_count + s->first_row + batch_size); } else { // warp 3 target_pos = min(s->nz_count, src_pos + batch_size); @@ -515,18 +514,18 @@ __global__ void __launch_bounds__(decode_block_size) // warp0 will decode the rep/def levels, warp1 will unpack a mini-batch of prefixes, warp 2 will // unpack a mini-batch of suffixes. warp3 waits one cycle for warps 0-2 to produce a batch, and // then stuffs values into the proper location in the output. - if (t < 32) { + if (t < warp_size) { // decode repetition and definition levels. // - update validity vectors // - updates offsets (for nested columns) // - produces non-NULL value indices in s->nz_idx for subsequent decoding gpuDecodeLevels(s, sb, target_pos, rep, def, t); - } else if (t < 64) { + } else if (t < 2 * warp_size) { // warp 1 prefix_db->decode_batch(); - } else if (t < 96) { + } else if (t < 3 * warp_size) { // warp 2 suffix_db->decode_batch(); diff --git a/cpp/src/io/parquet/parquet_gpu.hpp b/cpp/src/io/parquet/parquet_gpu.hpp index 12a5daecbd8..06cad416774 100644 --- a/cpp/src/io/parquet/parquet_gpu.hpp +++ b/cpp/src/io/parquet/parquet_gpu.hpp @@ -27,7 +27,6 @@ #include #include -#include #include #include #include From af9fb24b1dc2d36df01c39101eb300a0d2a690b1 Mon Sep 17 00:00:00 2001 From: seidl Date: Wed, 4 Oct 2023 16:08:55 -0700 Subject: [PATCH 09/39] add some documentation --- cpp/src/io/parquet/page_string_decode.cu | 20 ++++++++++++++------ 1 file changed, 14 insertions(+), 6 deletions(-) diff --git a/cpp/src/io/parquet/page_string_decode.cu b/cpp/src/io/parquet/page_string_decode.cu index 9a036d93734..2fbeea1bc91 100644 --- a/cpp/src/io/parquet/page_string_decode.cu +++ b/cpp/src/io/parquet/page_string_decode.cu @@ -463,12 +463,19 @@ __device__ size_t totalPlainEntriesSize(uint8_t const* data, /** * @brief Compute string size information for DELTA_BYTE_ARRAY encoded strings. * - * Called with 64 threads + * This traverses the packed prefix and suffix lengths, summing them to obtain the total + * number of bytes needed for the decoded string data. It also calculates an upper bound + * for the largest string length to obtain an upper bound on temporary space needed if + * rows will be skipped. + * + * Called with 64 threads. * * @param data Pointer to the start of the page data stream * @param end Pointer to the end of the page data stream * @param start_value Do not count values that occur before this index * @param end_value Do not count values that occur after this index + * @return A pair of `size_t` values representing the total string size and temp buffer size + * required for decoding */ __device__ thrust::pair totalDeltaByteArraySize(uint8_t const* data, uint8_t const* end, @@ -540,13 +547,14 @@ __device__ thrust::pair totalDeltaByteArraySize(uint8_t const* d cudf::detail::single_lane_block_sum_reduce(total_bytes); // sum up prefix and suffix max lengths - auto final_max = cudf::detail::single_lane_block_sum_reduce(max_len); + auto temp_bytes = + cudf::detail::single_lane_block_sum_reduce(max_len); if (t == 0) { // save enough for one mimi-block plus some extra to save the last_string - final_max *= db->values_per_mb + 1; + temp_bytes *= db->values_per_mb + 1; } - return {final_bytes, final_max}; + return {final_bytes, temp_bytes}; } /** @@ -680,14 +688,14 @@ __global__ void __launch_bounds__(delta_preproc_block_size) gpuComputeDeltaPageS uint8_t const* const end = s->data_end; auto const end_value = pp->end_val; - auto const [len, max] = totalDeltaByteArraySize(data, end, start_value, end_value); + auto const [len, temp_bytes] = totalDeltaByteArraySize(data, end, start_value, end_value); if (t == 0) { // TODO check for overflow pp->str_bytes = len; // only need temp space if we're skipping values - if (start_value > 0) { pp->temp_string_size = max; } + if (start_value > 0) { pp->temp_string_size = temp_bytes; } } } } From c8e39383f898355f8856d4f41d01586fb691ad44 Mon Sep 17 00:00:00 2001 From: seidl Date: Wed, 4 Oct 2023 16:33:25 -0700 Subject: [PATCH 10/39] more cleanup --- cpp/src/io/parquet/page_string_decode.cu | 54 +++++++----------------- 1 file changed, 15 insertions(+), 39 deletions(-) diff --git a/cpp/src/io/parquet/page_string_decode.cu b/cpp/src/io/parquet/page_string_decode.cu index 2fbeea1bc91..ae98a62e70a 100644 --- a/cpp/src/io/parquet/page_string_decode.cu +++ b/cpp/src/io/parquet/page_string_decode.cu @@ -27,10 +27,7 @@ #include -namespace cudf { -namespace io { -namespace parquet { -namespace gpu { +namespace cudf::io::parquet::gpu { namespace { @@ -601,17 +598,10 @@ __global__ void __launch_bounds__(preprocess_block_size) gpuComputeStringPageBou __shared__ rle_run rep_runs[rle_run_buffer_size]; rle_stream decoders[level_type::NUM_LEVEL_TYPES] = {{def_runs}, {rep_runs}}; + // setup page info - if (!setupLocalPageInfo( - s, - pp, - chunks, - min_row, - num_rows, - mask_filter{BitOr(decode_kernel_mask::STRING, decode_kernel_mask::DELTA_BYTE_ARRAY)}, - true)) { - return; - } + auto const mask = BitOr(decode_kernel_mask::STRING, decode_kernel_mask::DELTA_BYTE_ARRAY); + if (!setupLocalPageInfo(s, pp, chunks, min_row, num_rows, mask_filter{mask}, true)) { return; } bool const is_bounds_pg = is_bounds_page(s, min_row, num_rows, has_repetition); @@ -655,15 +645,8 @@ __global__ void __launch_bounds__(delta_preproc_block_size) gpuComputeDeltaPageS bool const has_repetition = chunks[pp->chunk_idx].max_level[level_type::REPETITION] > 0; // setup page info - if (!setupLocalPageInfo(s, - pp, - chunks, - min_row, - num_rows, - mask_filter{decode_kernel_mask::DELTA_BYTE_ARRAY}, - true)) { - return; - } + auto const mask = decode_kernel_mask::DELTA_BYTE_ARRAY; + if (!setupLocalPageInfo(s, pp, chunks, min_row, num_rows, mask_filter{mask}, true)) { return; } auto const start_value = pp->start_val; @@ -821,13 +804,9 @@ __global__ void __launch_bounds__(decode_block_size) int const lane_id = t % warp_size; [[maybe_unused]] null_count_back_copier _{s, t}; - if (!setupLocalPageInfo(s, - &pages[page_idx], - chunks, - min_row, - num_rows, - mask_filter{decode_kernel_mask::STRING}, - true)) { + auto const mask = decode_kernel_mask::STRING; + if (!setupLocalPageInfo( + s, &pages[page_idx], chunks, min_row, num_rows, mask_filter{mask}, true)) { return; } @@ -1003,8 +982,8 @@ void ComputePageStringSizes(cudf::detail::hostdevice_vector& pages, uint32_t kernel_mask, rmm::cuda_stream_view stream) { - dim3 dim_block(preprocess_block_size, 1); - dim3 dim_grid(pages.size(), 1); // 1 threadblock per page + dim3 const dim_block(preprocess_block_size, 1); + dim3 const dim_grid(pages.size(), 1); // 1 threadblock per page if (level_type_size == 1) { gpuComputeStringPageBounds <<>>(pages.device_ptr(), chunks, min_row, num_rows); @@ -1014,10 +993,10 @@ void ComputePageStringSizes(cudf::detail::hostdevice_vector& pages, } // kernel mask may contain other kernels we don't need to count - int count_mask = + int const count_mask = kernel_mask & BitOr(decode_kernel_mask::DELTA_BYTE_ARRAY, decode_kernel_mask::STRING); - int nkernels = std::bitset<32>(count_mask).count(); - auto streams = cudf::detail::fork_streams(stream, nkernels); + int const nkernels = std::bitset<32>(count_mask).count(); + auto const streams = cudf::detail::fork_streams(stream, nkernels); int s_idx = 0; if (BitAnd(kernel_mask, decode_kernel_mask::DELTA_BYTE_ARRAY) != 0) { @@ -1093,7 +1072,4 @@ void __host__ DecodeStringPageData(cudf::detail::hostdevice_vector& pa } } -} // namespace gpu -} // namespace parquet -} // namespace io -} // namespace cudf +} // namespace cudf::io::parquet::gpu From 0332923483808cc1984fe374f89bd8d45f5bf8cc Mon Sep 17 00:00:00 2001 From: seidl Date: Wed, 4 Oct 2023 16:42:37 -0700 Subject: [PATCH 11/39] more cleanup --- cpp/src/io/parquet/page_delta_decode.cu | 28 +++++++++---------------- 1 file changed, 10 insertions(+), 18 deletions(-) diff --git a/cpp/src/io/parquet/page_delta_decode.cu b/cpp/src/io/parquet/page_delta_decode.cu index 8353fe3d401..4522bb557a3 100644 --- a/cpp/src/io/parquet/page_delta_decode.cu +++ b/cpp/src/io/parquet/page_delta_decode.cu @@ -320,13 +320,9 @@ __global__ void __launch_bounds__(96) auto* const db = &db_state; [[maybe_unused]] null_count_back_copier _{s, t}; - if (!setupLocalPageInfo(s, - &pages[page_idx], - chunks, - min_row, - num_rows, - mask_filter{decode_kernel_mask::DELTA_BINARY}, - true)) { + auto const mask = decode_kernel_mask::DELTA_BINARY; + if (!setupLocalPageInfo( + s, &pages[page_idx], chunks, min_row, num_rows, mask_filter{mask}, true)) { return; } @@ -455,13 +451,9 @@ __global__ void __launch_bounds__(decode_block_size) auto* const dba = &db_state; [[maybe_unused]] null_count_back_copier _{s, t}; - if (!setupLocalPageInfo(s, - &pages[page_idx], - chunks, - min_row, - num_rows, - mask_filter{decode_kernel_mask::DELTA_BYTE_ARRAY}, - true)) { + auto const mask = decode_kernel_mask::DELTA_BYTE_ARRAY; + if (!setupLocalPageInfo( + s, &pages[page_idx], chunks, min_row, num_rows, mask_filter{mask}, true)) { return; } @@ -532,7 +524,7 @@ __global__ void __launch_bounds__(decode_block_size) } else if (src_pos < target_pos) { // warp 3 - int nproc = min(batch_size, s->page.end_val - string_pos); + int const nproc = min(batch_size, s->page.end_val - string_pos); strings_data += use_char_ll ? dba->calculate_string_values_cp(strings_data, string_pos, nproc, lane_id) : dba->calculate_string_values(strings_data, string_pos, nproc, lane_id); @@ -547,7 +539,7 @@ __global__ void __launch_bounds__(decode_block_size) if (!has_repetition) { dst_pos -= s->first_row; } if (dst_pos >= 0 && sp < target_pos) { - auto offptr = + auto const offptr = reinterpret_cast(nesting_info_base[leaf_level_index].data_out) + dst_pos; auto const src_idx = rolling_index(sp + skipped_leaf_values); *offptr = prefix_db->value[src_idx] + suffix_db->value[src_idx]; @@ -617,8 +609,8 @@ void __host__ DecodeDeltaByteArray(cudf::detail::hostdevice_vector& pa { CUDF_EXPECTS(pages.size() > 0, "There is no page to decode"); - dim3 dim_block(decode_block_size, 1); - dim3 dim_grid(pages.size(), 1); // 1 threadblock per page + dim3 const dim_block(decode_block_size, 1); + dim3 const dim_grid(pages.size(), 1); // 1 threadblock per page if (level_type_size == 1) { gpuDecodeDeltaByteArray<<>>( From f5f2bd5da637acce08f35a0ee196518c4655b297 Mon Sep 17 00:00:00 2001 From: seidl Date: Wed, 4 Oct 2023 16:47:09 -0700 Subject: [PATCH 12/39] a little more cleanup --- cpp/src/io/parquet/page_data.cu | 10 +++------- 1 file changed, 3 insertions(+), 7 deletions(-) diff --git a/cpp/src/io/parquet/page_data.cu b/cpp/src/io/parquet/page_data.cu index 9414c6c71cf..c0b92752240 100644 --- a/cpp/src/io/parquet/page_data.cu +++ b/cpp/src/io/parquet/page_data.cu @@ -452,13 +452,9 @@ __global__ void __launch_bounds__(decode_block_size) int out_thread0; [[maybe_unused]] null_count_back_copier _{s, t}; - if (!setupLocalPageInfo(s, - &pages[page_idx], - chunks, - min_row, - num_rows, - mask_filter{decode_kernel_mask::GENERAL}, - true)) { + auto const mask = decode_kernel_mask::GENERAL; + if (!setupLocalPageInfo( + s, &pages[page_idx], chunks, min_row, num_rows, mask_filter{mask}, true)) { return; } From 2c306462b7926cfb9cc7410a8e4aa5d9db4fc716 Mon Sep 17 00:00:00 2001 From: seidl Date: Mon, 9 Oct 2023 14:11:23 -0700 Subject: [PATCH 13/39] clean up some docstrings and move where str_bytes is initialized --- cpp/src/io/parquet/page_string_decode.cu | 37 ++++++++++++++---------- 1 file changed, 22 insertions(+), 15 deletions(-) diff --git a/cpp/src/io/parquet/page_string_decode.cu b/cpp/src/io/parquet/page_string_decode.cu index 93c2080a1d6..b4cb73a07ff 100644 --- a/cpp/src/io/parquet/page_string_decode.cu +++ b/cpp/src/io/parquet/page_string_decode.cu @@ -555,12 +555,13 @@ __device__ thrust::pair totalDeltaByteArraySize(uint8_t const* d } /** - * @brief Kernel for computing string page output size information. + * @brief Kernel for computing string page bounds information. * - * String columns need accurate data size information to preallocate memory in the column buffer to - * store the char data. This calls a kernel to calculate information needed by the string decoding - * kernel. On exit, the `str_bytes`, `num_nulls`, and `num_valids` fields of the PageInfo struct - * are updated. This call ignores non-string columns. + * This kernel traverses the repetition and definition level data to determ start and end values + * for pages with string-like data. Also calculates the number of null and valid values in the + * page. Does nothing if the page mask is neither `STRING` nor `DELTA_BYTE_ARRAY`. On exit the + * `num_nulls`, `num_valids`, `start_val` and `end_val` fields of the `PageInfo` struct will be + * populated. Also fills in the `temp_string_size` field if rows are to be skipped. * * @param pages All pages to be decoded * @param chunks All chunks to be decoded @@ -582,8 +583,6 @@ __global__ void __launch_bounds__(preprocess_block_size) gpuComputeStringPageBou if (t == 0) { s->page.num_nulls = 0; s->page.num_valids = 0; - // reset str_bytes to 0 in case it's already been calculated - pp->str_bytes = 0; } // whether or not we have repetition levels (lists) @@ -621,10 +620,8 @@ __global__ void __launch_bounds__(preprocess_block_size) gpuComputeStringPageBou /** * @brief Kernel for computing string page output size information for delta_byte_array encoding. * - * String columns need accurate data size information to preallocate memory in the column buffer to - * store the char data. This calls a kernel to calculate information needed by the string decoding - * kernel. On exit, the `str_bytes`, `num_nulls`, and `num_valids` fields of the PageInfo struct - * are updated. This call ignores non-string columns. + * This call ignores columns that are not DELTA_BYTE_ARRAY encoded. On exit the `str_bytes` field + * of the `PageInfo` struct will be populated. * * @param pages All pages to be decoded * @param chunks All chunks to be decoded @@ -648,6 +645,12 @@ __global__ void __launch_bounds__(delta_preproc_block_size) gpuComputeDeltaPageS auto const mask = decode_kernel_mask::DELTA_BYTE_ARRAY; if (!setupLocalPageInfo(s, pp, chunks, min_row, num_rows, mask_filter{mask}, true)) { return; } + if (t == 0) { + // reset str_bytes to 0 in case it's already been calculated + // TODO: need to rethink this once str_bytes is in the statistics + pp->str_bytes = 0; + } + auto const start_value = pp->start_val; // if data size is known, can short circuit here @@ -686,10 +689,8 @@ __global__ void __launch_bounds__(delta_preproc_block_size) gpuComputeDeltaPageS /** * @brief Kernel for computing string page output size information. * - * String columns need accurate data size information to preallocate memory in the column buffer to - * store the char data. This calls a kernel to calculate information needed by the string decoding - * kernel. On exit, the `str_bytes`, `num_nulls`, and `num_valids` fields of the PageInfo struct - * are updated. This call ignores non-string columns. + * This call ignores non-string columns. On exit the `str_bytes` field of the `PageInfo` struct will + * be populated. * * @param pages All pages to be decoded * @param chunks All chunks to be decoded @@ -715,6 +716,12 @@ __global__ void __launch_bounds__(preprocess_block_size) gpuComputePageStringSiz return; } + if (t == 0) { + // reset str_bytes to 0 in case it's already been calculated + // TODO: need to rethink this once str_bytes is in the statistics + pp->str_bytes = 0; + } + bool const is_bounds_pg = is_bounds_page(s, min_row, num_rows, has_repetition); auto const& col = s->col; From c2dbb47b23b83547a755556aed35f498b99e1d66 Mon Sep 17 00:00:00 2001 From: seidl Date: Mon, 9 Oct 2023 14:41:10 -0700 Subject: [PATCH 14/39] revert move of str_bytes reset...broke chunked reads --- cpp/src/io/parquet/page_string_decode.cu | 15 +++------------ 1 file changed, 3 insertions(+), 12 deletions(-) diff --git a/cpp/src/io/parquet/page_string_decode.cu b/cpp/src/io/parquet/page_string_decode.cu index b4cb73a07ff..3520316b955 100644 --- a/cpp/src/io/parquet/page_string_decode.cu +++ b/cpp/src/io/parquet/page_string_decode.cu @@ -583,6 +583,9 @@ __global__ void __launch_bounds__(preprocess_block_size) gpuComputeStringPageBou if (t == 0) { s->page.num_nulls = 0; s->page.num_valids = 0; + // reset str_bytes to 0 in case it's already been calculated (esp needed for chunked reads). + // TODO: need to rethink this once str_bytes is in the statistics + pp->str_bytes = 0; } // whether or not we have repetition levels (lists) @@ -645,12 +648,6 @@ __global__ void __launch_bounds__(delta_preproc_block_size) gpuComputeDeltaPageS auto const mask = decode_kernel_mask::DELTA_BYTE_ARRAY; if (!setupLocalPageInfo(s, pp, chunks, min_row, num_rows, mask_filter{mask}, true)) { return; } - if (t == 0) { - // reset str_bytes to 0 in case it's already been calculated - // TODO: need to rethink this once str_bytes is in the statistics - pp->str_bytes = 0; - } - auto const start_value = pp->start_val; // if data size is known, can short circuit here @@ -716,12 +713,6 @@ __global__ void __launch_bounds__(preprocess_block_size) gpuComputePageStringSiz return; } - if (t == 0) { - // reset str_bytes to 0 in case it's already been calculated - // TODO: need to rethink this once str_bytes is in the statistics - pp->str_bytes = 0; - } - bool const is_bounds_pg = is_bounds_page(s, min_row, num_rows, has_repetition); auto const& col = s->col; From 71b979cd5689a86cf26fbf19f4b3880e8a73387c Mon Sep 17 00:00:00 2001 From: seidl Date: Wed, 18 Oct 2023 13:02:33 -0700 Subject: [PATCH 15/39] remove redundant comment --- cpp/src/io/parquet/parquet_gpu.hpp | 1 - 1 file changed, 1 deletion(-) diff --git a/cpp/src/io/parquet/parquet_gpu.hpp b/cpp/src/io/parquet/parquet_gpu.hpp index 5d1105e4623..4429f889080 100644 --- a/cpp/src/io/parquet/parquet_gpu.hpp +++ b/cpp/src/io/parquet/parquet_gpu.hpp @@ -116,7 +116,6 @@ constexpr uint32_t BitAnd(T1 a, T2 b) return static_cast(a) & static_cast(b); } -// helpers to do bit operations on scoped enums template ::value and std::is_same_v) or From 46d179a97bc0c4fa3c0d7cef683ddac05db65dc7 Mon Sep 17 00:00:00 2001 From: seidl Date: Fri, 20 Oct 2023 14:14:49 -0700 Subject: [PATCH 16/39] fix test for supported encodings lost in merge --- cpp/src/io/parquet/parquet_gpu.hpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/cpp/src/io/parquet/parquet_gpu.hpp b/cpp/src/io/parquet/parquet_gpu.hpp index aca3c51954b..52856d237ec 100644 --- a/cpp/src/io/parquet/parquet_gpu.hpp +++ b/cpp/src/io/parquet/parquet_gpu.hpp @@ -65,7 +65,8 @@ constexpr bool is_supported_encoding(Encoding enc) case Encoding::PLAIN_DICTIONARY: case Encoding::RLE: case Encoding::RLE_DICTIONARY: - case Encoding::DELTA_BINARY_PACKED: return true; + case Encoding::DELTA_BINARY_PACKED: + case Encoding::DELTA_BYTE_ARRAY: return true; default: return false; } } From 29df5bd0936cfdd59571be8de37c9b2f300110f1 Mon Sep 17 00:00:00 2001 From: seidl Date: Mon, 30 Oct 2023 12:45:48 -0700 Subject: [PATCH 17/39] move description of DELTA_BYTE_ARRAY to where it belongs. add some clarification and rename suffix_data. --- cpp/src/io/parquet/delta_binary.cuh | 6 ----- cpp/src/io/parquet/page_delta_decode.cu | 32 +++++++++++++++---------- 2 files changed, 19 insertions(+), 19 deletions(-) diff --git a/cpp/src/io/parquet/delta_binary.cuh b/cpp/src/io/parquet/delta_binary.cuh index 92bf1f330fd..ecaf0eb4fbe 100644 --- a/cpp/src/io/parquet/delta_binary.cuh +++ b/cpp/src/io/parquet/delta_binary.cuh @@ -39,12 +39,6 @@ namespace cudf::io::parquet::detail { // per mini-block. While encoding, the lowest delta value is subtracted from all the deltas in the // block to ensure that all encoded values are positive. The deltas for each mini-block are bit // packed using the same encoding as the RLE/Bit-Packing Hybrid encoder. -// -// DELTA_BYTE_ARRAY encoding (incremental encoding or front compression), is used for BYTE_ARRAY -// columns. For each element in a sequence of strings, a prefix length from the preceding string -// and a suffix is stored. The prefix lengths are DELTA_BINARY_PACKED encoded. The suffixes are -// encoded with DELTA_LENGTH_BYTE_ARRAY encoding, which is a DELTA_BINARY_PACKED list of suffix -// lengths, followed by the concatenated suffix data. // we decode one mini-block at a time. max mini-block size seen is 64. constexpr int delta_rolling_buf_size = 128; diff --git a/cpp/src/io/parquet/page_delta_decode.cu b/cpp/src/io/parquet/page_delta_decode.cu index bb9ccb64129..0d86142b985 100644 --- a/cpp/src/io/parquet/page_delta_decode.cu +++ b/cpp/src/io/parquet/page_delta_decode.cu @@ -29,14 +29,18 @@ namespace { constexpr int decode_block_size = 128; +// DELTA_BYTE_ARRAY encoding (incremental encoding or front compression), is used for BYTE_ARRAY +// columns. For each element in a sequence of strings, a prefix length from the preceding string +// and a suffix is stored. The prefix lengths are DELTA_BINARY_PACKED encoded. The suffixes are +// encoded with DELTA_LENGTH_BYTE_ARRAY encoding, which is a DELTA_BINARY_PACKED list of suffix +// lengths, followed by the concatenated suffix data. struct delta_byte_array_decoder { - uint8_t const* last_string; - uint8_t const* suffix_data; + uint8_t const* last_string; // pointer to last decoded string...needed for its prefix + uint8_t const* suffix_char_data; // pointer to the start of character data - // used when skipping values - uint8_t* temp_buf; - uint32_t start_val; // decoded strings up to this index will be dumped to temp_buf - uint32_t last_string_len; + uint8_t* temp_buf; // buffer used when skipping values + uint32_t start_val; // decoded strings up to this index will be dumped to temp_buf + uint32_t last_string_len; // length of the last decoded string delta_binary_decoder prefixes; // state of decoder for prefix lengths delta_binary_decoder suffixes; // state of decoder for suffix lengths @@ -45,7 +49,7 @@ struct delta_byte_array_decoder { __device__ void init(uint8_t const* start, uint8_t const* end, uint32_t start_idx, uint8_t* temp) { auto const* suffix_start = prefixes.find_end_of_block(start, end); - suffix_data = suffixes.find_end_of_block(suffix_start, end); + suffix_char_data = suffixes.find_end_of_block(suffix_start, end); last_string = nullptr; temp_buf = temp; start_val = start_idx; @@ -154,7 +158,7 @@ struct delta_byte_array_decoder { auto const so_ptr = out + string_off; // copy suffixes into string data - if (ln_idx < end) { memcpy(so_ptr + prefix_len, suffix_data + suffix_off, suffix_len); } + if (ln_idx < end) { memcpy(so_ptr + prefix_len, suffix_char_data + suffix_off, suffix_len); } __syncwarp(); // copy prefixes into string data. @@ -166,8 +170,8 @@ struct delta_byte_array_decoder { // set last_string to this lane's string last_string = out + string_off; last_string_len = string_len; - // and consume used suffix_data - suffix_data += suffix_off + suffix_len; + // and consume used suffix_char_data + suffix_char_data += suffix_off + suffix_len; } return warp_total; @@ -229,10 +233,12 @@ struct delta_byte_array_decoder { // for longer strings use a 4-byte version stolen from gather_chars_fn_string_parallel. if (string_len > 64) { if (prefix_len > 0) { wideStrcpy(so_ptr, last_string, prefix_len, lane_id); } - if (suffix_len > 0) { wideStrcpy(so_ptr + prefix_len, suffix_data, suffix_len, lane_id); } + if (suffix_len > 0) { + wideStrcpy(so_ptr + prefix_len, suffix_char_data, suffix_len, lane_id); + } } else { for (int i = lane_id; i < string_len; i += warp_size) { - so_ptr[i] = i < prefix_len ? last_string[i] : suffix_data[i - prefix_len]; + so_ptr[i] = i < prefix_len ? last_string[i] : suffix_char_data[i - prefix_len]; } } __syncwarp(); @@ -242,7 +248,7 @@ struct delta_byte_array_decoder { if (lane_id == 0) { last_string = so_ptr; last_string_len = string_len; - suffix_data += suffix_len; + suffix_char_data += suffix_len; if (idx == start_val - 1) { so_ptr = strings_out; } else { From 21d7f9b95999e4a03aed8f5bd00dd266056988e0 Mon Sep 17 00:00:00 2001 From: Ed Seidl Date: Tue, 31 Oct 2023 21:53:38 -0700 Subject: [PATCH 18/39] typo caught by review Co-authored-by: Vukasin Milovanovic --- cpp/src/io/parquet/page_string_decode.cu | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cpp/src/io/parquet/page_string_decode.cu b/cpp/src/io/parquet/page_string_decode.cu index d5edf98980e..7df9698c394 100644 --- a/cpp/src/io/parquet/page_string_decode.cu +++ b/cpp/src/io/parquet/page_string_decode.cu @@ -547,7 +547,7 @@ __device__ thrust::pair totalDeltaByteArraySize(uint8_t const* d auto temp_bytes = cudf::detail::single_lane_block_sum_reduce(max_len); if (t == 0) { - // save enough for one mimi-block plus some extra to save the last_string + // save enough for one mini-block plus some extra to save the last_string temp_bytes *= db->values_per_mb + 1; } From ebef98d1ff3f0b529bb174ae328e62b408d2b682 Mon Sep 17 00:00:00 2001 From: Ed Seidl Date: Tue, 31 Oct 2023 22:08:13 -0700 Subject: [PATCH 19/39] update calculation of temp_bytes --- cpp/src/io/parquet/page_string_decode.cu | 9 +++------ 1 file changed, 3 insertions(+), 6 deletions(-) diff --git a/cpp/src/io/parquet/page_string_decode.cu b/cpp/src/io/parquet/page_string_decode.cu index d5edf98980e..96bf2997a79 100644 --- a/cpp/src/io/parquet/page_string_decode.cu +++ b/cpp/src/io/parquet/page_string_decode.cu @@ -543,13 +543,10 @@ __device__ thrust::pair totalDeltaByteArraySize(uint8_t const* d auto const final_bytes = cudf::detail::single_lane_block_sum_reduce(total_bytes); - // sum up prefix and suffix max lengths + // Sum up prefix and suffix max lengths to get a max possible string length. Multiply that + // by the number of strings in a mini-block, plus one to save the last string. auto temp_bytes = - cudf::detail::single_lane_block_sum_reduce(max_len); - if (t == 0) { - // save enough for one mimi-block plus some extra to save the last_string - temp_bytes *= db->values_per_mb + 1; - } + cudf::detail::single_lane_block_sum_reduce(max_len) * (db->values_per_mb + 1); return {final_bytes, temp_bytes}; } From cb2d7ce1a9a0aa1884b3d21c60c875dc42b30146 Mon Sep 17 00:00:00 2001 From: Ed Seidl Date: Tue, 31 Oct 2023 22:14:49 -0700 Subject: [PATCH 20/39] fix comment Co-authored-by: Vukasin Milovanovic --- cpp/src/io/parquet/page_delta_decode.cu | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cpp/src/io/parquet/page_delta_decode.cu b/cpp/src/io/parquet/page_delta_decode.cu index 0d86142b985..67b0e6af3a2 100644 --- a/cpp/src/io/parquet/page_delta_decode.cu +++ b/cpp/src/io/parquet/page_delta_decode.cu @@ -121,7 +121,7 @@ struct delta_byte_array_decoder { // calculate a mini-batch of string values, writing the results to // `strings_out`. starting at global index `start_idx` and decoding // up to `num_values` strings. - // called by all threads in a warp. used for strings < 32 chars. + // called by all threads in a warp. used for strings <= 32 chars. // returns number of bytes written __device__ size_t calculate_string_values(uint8_t* strings_out, uint32_t start_idx, From 5c1549b23a747e1dacdc4902ade85824963c7272 Mon Sep 17 00:00:00 2001 From: Ed Seidl Date: Tue, 31 Oct 2023 23:02:48 -0700 Subject: [PATCH 21/39] add some explanation for what is going on in ComputePageStringSizes --- cpp/src/io/parquet/page_string_decode.cu | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/cpp/src/io/parquet/page_string_decode.cu b/cpp/src/io/parquet/page_string_decode.cu index 96bf2997a79..6b502486f60 100644 --- a/cpp/src/io/parquet/page_string_decode.cu +++ b/cpp/src/io/parquet/page_string_decode.cu @@ -950,6 +950,9 @@ __global__ void __launch_bounds__(decode_block_size) if (t == 0 and s->error != 0) { set_error(s->error, error_code); } } +// Functor used to set the `temp_string_buf` pointer for each page. `data` points to a buffer +// to be used when skipping rows in the delta_byte_array decoder. Given a page and an offset, +// set the page's `temp_string_buf` to be `data + offset`. struct page_tform_functor { void* const data; @@ -1011,6 +1014,7 @@ void ComputePageStringSizes(cudf::detail::hostdevice_vector& pages, }); if (need_sizes) { + // sum up all of the temp_string_sizes auto const page_sizes = [] __device__(PageInfo const& page) { return page.temp_string_size; }; int64_t total_size = thrust::transform_reduce(rmm::exec_policy(stream), pages.d_begin(), @@ -1019,6 +1023,8 @@ void ComputePageStringSizes(cudf::detail::hostdevice_vector& pages, 0L, thrust::plus{}); + // now do an exclusive scan over the temp_string_sizes to get offsets for each + // page's chunk of the temp buffer rmm::device_uvector page_string_offsets(pages.size(), stream); thrust::transform_exclusive_scan(rmm::exec_policy(stream), pages.d_begin(), @@ -1028,8 +1034,10 @@ void ComputePageStringSizes(cudf::detail::hostdevice_vector& pages, 0L, thrust::plus{}); + // allocate the temp space temp_string_buf = rmm::device_buffer(total_size, stream); + // now use the offsets array to set each page's temp_string_buf pointers thrust::transform(rmm::exec_policy(stream), pages.d_begin(), pages.d_end(), From bad9f6acd8c00be2e1bdedf33f1da9e73c2a03d7 Mon Sep 17 00:00:00 2001 From: seidl Date: Wed, 1 Nov 2023 08:39:14 -0700 Subject: [PATCH 22/39] use device_uvector --- cpp/src/io/parquet/page_string_decode.cu | 11 ++++++----- cpp/src/io/parquet/parquet_gpu.hpp | 2 +- cpp/src/io/parquet/reader_impl.cpp | 2 +- 3 files changed, 8 insertions(+), 7 deletions(-) diff --git a/cpp/src/io/parquet/page_string_decode.cu b/cpp/src/io/parquet/page_string_decode.cu index 6b502486f60..45ddd436b2a 100644 --- a/cpp/src/io/parquet/page_string_decode.cu +++ b/cpp/src/io/parquet/page_string_decode.cu @@ -546,7 +546,8 @@ __device__ thrust::pair totalDeltaByteArraySize(uint8_t const* d // Sum up prefix and suffix max lengths to get a max possible string length. Multiply that // by the number of strings in a mini-block, plus one to save the last string. auto temp_bytes = - cudf::detail::single_lane_block_sum_reduce(max_len) * (db->values_per_mb + 1); + cudf::detail::single_lane_block_sum_reduce(max_len) * + (db->values_per_mb + 1); return {final_bytes, temp_bytes}; } @@ -954,11 +955,11 @@ __global__ void __launch_bounds__(decode_block_size) // to be used when skipping rows in the delta_byte_array decoder. Given a page and an offset, // set the page's `temp_string_buf` to be `data + offset`. struct page_tform_functor { - void* const data; + uint8_t* const data; __device__ PageInfo operator()(PageInfo& page, int64_t offset) { - if (page.temp_string_size != 0) { page.temp_string_buf = static_cast(data) + offset; } + if (page.temp_string_size != 0) { page.temp_string_buf = data + offset; } return page; } }; @@ -970,7 +971,7 @@ struct page_tform_functor { */ void ComputePageStringSizes(cudf::detail::hostdevice_vector& pages, cudf::detail::hostdevice_vector const& chunks, - rmm::device_buffer& temp_string_buf, + rmm::device_uvector& temp_string_buf, size_t min_row, size_t num_rows, int level_type_size, @@ -1035,7 +1036,7 @@ void ComputePageStringSizes(cudf::detail::hostdevice_vector& pages, thrust::plus{}); // allocate the temp space - temp_string_buf = rmm::device_buffer(total_size, stream); + temp_string_buf.resize(total_size, stream); // now use the offsets array to set each page's temp_string_buf pointers thrust::transform(rmm::exec_policy(stream), diff --git a/cpp/src/io/parquet/parquet_gpu.hpp b/cpp/src/io/parquet/parquet_gpu.hpp index 3b3b7aed190..dbc6ecdd63c 100644 --- a/cpp/src/io/parquet/parquet_gpu.hpp +++ b/cpp/src/io/parquet/parquet_gpu.hpp @@ -627,7 +627,7 @@ void ComputePageSizes(cudf::detail::hostdevice_vector& pages, */ void ComputePageStringSizes(cudf::detail::hostdevice_vector& pages, cudf::detail::hostdevice_vector const& chunks, - rmm::device_buffer& temp_string_buf, + rmm::device_uvector& temp_string_buf, size_t min_row, size_t num_rows, int level_type_size, diff --git a/cpp/src/io/parquet/reader_impl.cpp b/cpp/src/io/parquet/reader_impl.cpp index ba14e28a05e..6e799424d01 100644 --- a/cpp/src/io/parquet/reader_impl.cpp +++ b/cpp/src/io/parquet/reader_impl.cpp @@ -37,7 +37,7 @@ void reader::impl::decode_page_data(size_t skip_rows, size_t num_rows) // temporary space for DELTA_BYTE_ARRAY decoding. this only needs to live until // gpu::DecodeDeltaByteArray returns. - rmm::device_buffer delta_temp_buf; + rmm::device_uvector delta_temp_buf(0, _stream); // Should not reach here if there is no page data. CUDF_EXPECTS(pages.size() > 0, "There is no page to decode"); From 75e23fcdc111373405c406358845768a54920e7f Mon Sep 17 00:00:00 2001 From: seidl Date: Wed, 1 Nov 2023 09:00:51 -0700 Subject: [PATCH 23/39] implement a TODO --- cpp/src/io/parquet/page_delta_decode.cu | 13 +++++++++---- cpp/src/io/parquet/parquet_gpu.hpp | 1 + 2 files changed, 10 insertions(+), 4 deletions(-) diff --git a/cpp/src/io/parquet/page_delta_decode.cu b/cpp/src/io/parquet/page_delta_decode.cu index 67b0e6af3a2..5a4ef36e071 100644 --- a/cpp/src/io/parquet/page_delta_decode.cu +++ b/cpp/src/io/parquet/page_delta_decode.cu @@ -271,7 +271,7 @@ struct delta_byte_array_decoder { // is this even necessary? return if asking to skip the whole block. if (start_val >= prefixes.num_encoded_values(true)) { return; } - // TODO: this assumes prefixes and suffixes will have the same parameters + // prefixes and suffixes will have the same parameters (it's checked earlier) auto const batch_size = prefixes.values_per_mb; uint32_t skip_pos = 0; @@ -462,8 +462,6 @@ __global__ void __launch_bounds__(decode_block_size) bool const has_repetition = s->col.max_level[level_type::REPETITION] > 0; - // TODO(ets) assert string_data != nullptr - // choose a character parallel string copy when the average string is longer than a warp auto const use_char_ll = (s->page.str_bytes / s->page.num_valids) > cudf::detail::warp_size; @@ -482,11 +480,18 @@ __global__ void __launch_bounds__(decode_block_size) } __syncthreads(); + // assert that prefix and suffix have same mini-block size + if (prefix_db->values_per_mb != suffix_db->values_per_mb or + prefix_db->block_size != suffix_db->block_size or + prefix_db->value_count != suffix_db->value_count) { + set_error(static_cast(decode_error::DELTA_PARAM_MISMATCH), error_code); + return; + } + // pointer to location to output final strings int const leaf_level_index = s->col.max_nesting_depth - 1; auto strings_data = nesting_info_base[leaf_level_index].string_out; - // TODO(ets) assert that prefix and suffix have same mini-block size auto const batch_size = prefix_db->values_per_mb; // if this is a bounds page and nested, then we need to skip up front. non-nested will work diff --git a/cpp/src/io/parquet/parquet_gpu.hpp b/cpp/src/io/parquet/parquet_gpu.hpp index dbc6ecdd63c..d36cf646095 100644 --- a/cpp/src/io/parquet/parquet_gpu.hpp +++ b/cpp/src/io/parquet/parquet_gpu.hpp @@ -95,6 +95,7 @@ enum class decode_error : int32_t { INVALID_DATA_TYPE = 0x10, EMPTY_PAGE = 0x20, INVALID_DICT_WIDTH = 0x40, + DELTA_PARAM_MISMATCH = 0x80, }; /** From 300f492718f592ecd19884e8421305df43d18407 Mon Sep 17 00:00:00 2001 From: seidl Date: Wed, 1 Nov 2023 09:04:05 -0700 Subject: [PATCH 24/39] use exec_policy_nosync when calculating string sizs --- cpp/src/io/parquet/page_string_decode.cu | 4 ++-- cpp/src/io/parquet/reader_impl_preprocess.cu | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/cpp/src/io/parquet/page_string_decode.cu b/cpp/src/io/parquet/page_string_decode.cu index 45ddd436b2a..2a09a6aeabe 100644 --- a/cpp/src/io/parquet/page_string_decode.cu +++ b/cpp/src/io/parquet/page_string_decode.cu @@ -1027,7 +1027,7 @@ void ComputePageStringSizes(cudf::detail::hostdevice_vector& pages, // now do an exclusive scan over the temp_string_sizes to get offsets for each // page's chunk of the temp buffer rmm::device_uvector page_string_offsets(pages.size(), stream); - thrust::transform_exclusive_scan(rmm::exec_policy(stream), + thrust::transform_exclusive_scan(rmm::exec_policy_nosync(stream), pages.d_begin(), pages.d_end(), page_string_offsets.begin(), @@ -1039,7 +1039,7 @@ void ComputePageStringSizes(cudf::detail::hostdevice_vector& pages, temp_string_buf.resize(total_size, stream); // now use the offsets array to set each page's temp_string_buf pointers - thrust::transform(rmm::exec_policy(stream), + thrust::transform(rmm::exec_policy_nosync(stream), pages.d_begin(), pages.d_end(), page_string_offsets.begin(), diff --git a/cpp/src/io/parquet/reader_impl_preprocess.cu b/cpp/src/io/parquet/reader_impl_preprocess.cu index 80a4d00a5a2..0bc492546e9 100644 --- a/cpp/src/io/parquet/reader_impl_preprocess.cu +++ b/cpp/src/io/parquet/reader_impl_preprocess.cu @@ -1416,7 +1416,7 @@ std::vector reader::impl::calculate_page_string_offsets() page_index.begin(), page_to_string_size{pages.device_ptr(), chunks.device_ptr()}); // do scan by key to calculate string offsets for each page - thrust::exclusive_scan_by_key(rmm::exec_policy(_stream), + thrust::exclusive_scan_by_key(rmm::exec_policy_nosync(_stream), page_keys.begin(), page_keys.end(), val_iter, @@ -1424,7 +1424,7 @@ std::vector reader::impl::calculate_page_string_offsets() // now sum up page sizes rmm::device_uvector reduce_keys(col_sizes.size(), _stream); - thrust::reduce_by_key(rmm::exec_policy(_stream), + thrust::reduce_by_key(rmm::exec_policy_nosync(_stream), page_keys.begin(), page_keys.end(), val_iter, From fe7e5a2c49079b618dce371f34505ae024810284 Mon Sep 17 00:00:00 2001 From: seidl Date: Tue, 7 Nov 2023 13:23:26 -0800 Subject: [PATCH 25/39] fix typo --- cpp/src/io/parquet/page_string_decode.cu | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cpp/src/io/parquet/page_string_decode.cu b/cpp/src/io/parquet/page_string_decode.cu index 2a09a6aeabe..af508f9ebf8 100644 --- a/cpp/src/io/parquet/page_string_decode.cu +++ b/cpp/src/io/parquet/page_string_decode.cu @@ -555,7 +555,7 @@ __device__ thrust::pair totalDeltaByteArraySize(uint8_t const* d /** * @brief Kernel for computing string page bounds information. * - * This kernel traverses the repetition and definition level data to determ start and end values + * This kernel traverses the repetition and definition level data to determine start and end values * for pages with string-like data. Also calculates the number of null and valid values in the * page. Does nothing if the page mask is neither `STRING` nor `DELTA_BYTE_ARRAY`. On exit the * `num_nulls`, `num_valids`, `start_val` and `end_val` fields of the `PageInfo` struct will be From e9baf16f1b2fcd58dbf4155a42068b5d633f9621 Mon Sep 17 00:00:00 2001 From: seidl Date: Wed, 8 Nov 2023 11:22:03 -0800 Subject: [PATCH 26/39] add delta binary roundtrip test --- python/cudf/cudf/tests/test_parquet.py | 40 ++++++++++++++++++++++++++ 1 file changed, 40 insertions(+) diff --git a/python/cudf/cudf/tests/test_parquet.py b/python/cudf/cudf/tests/test_parquet.py index 4438dfa5344..72cc41ef2ea 100644 --- a/python/cudf/cudf/tests/test_parquet.py +++ b/python/cudf/cudf/tests/test_parquet.py @@ -1341,6 +1341,46 @@ def test_delta_binary(nrows, add_nulls, tmpdir): assert_eq(cdf, pcdf) +@pytest.mark.parametrize("nrows", [1, 100000]) +@pytest.mark.parametrize("add_nulls", [True, False]) +def test_delta_byte_array_roundtrip(nrows, add_nulls, tmpdir): + null_frequency = 0.25 if add_nulls else 0 + + # Create a pandas dataframe with random data of mixed lengths + test_pdf = dg.rand_dataframe( + dtypes_meta=[ + { + "dtype": "str", + "null_frequency": null_frequency, + "cardinality": nrows, + "max_string_length": 10, + }, + { + "dtype": "str", + "null_frequency": null_frequency, + "cardinality": nrows, + "max_string_length": 100, + }, + ], + rows=nrows, + seed=0, + use_threads=False, + ).to_pandas() + + pdf_fname = tmpdir.join("pdfdeltaba.parquet") + test_pdf.to_parquet( + pdf_fname, + version="2.6", + column_encoding="DELTA_BYTE_ARRAY", + data_page_version="2.0", + engine="pyarrow", + use_dictionary=False, + ) + cdf = cudf.read_parquet(pdf_fname) + pcdf = cudf.from_pandas(test_pdf) + assert_eq(cdf, pcdf) + + @pytest.mark.parametrize( "data", [ From 4e1160a1db181970d76c090c97dd0d455a68ea24 Mon Sep 17 00:00:00 2001 From: seidl Date: Wed, 8 Nov 2023 14:36:47 -0800 Subject: [PATCH 27/39] test a larger range of rows around critical numbers for the delta decoder --- python/cudf/cudf/tests/test_parquet.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/python/cudf/cudf/tests/test_parquet.py b/python/cudf/cudf/tests/test_parquet.py index 72cc41ef2ea..988863f951d 100644 --- a/python/cudf/cudf/tests/test_parquet.py +++ b/python/cudf/cudf/tests/test_parquet.py @@ -1341,7 +1341,9 @@ def test_delta_binary(nrows, add_nulls, tmpdir): assert_eq(cdf, pcdf) -@pytest.mark.parametrize("nrows", [1, 100000]) +@pytest.mark.parametrize( + "nrows", [1, 2, 23, 32, 33, 34, 64, 65, 66, 128, 129, 130, 20000, 100000] +) @pytest.mark.parametrize("add_nulls", [True, False]) def test_delta_byte_array_roundtrip(nrows, add_nulls, tmpdir): null_frequency = 0.25 if add_nulls else 0 From 734023b147ba2042222c05341ab464885c82ef30 Mon Sep 17 00:00:00 2001 From: seidl Date: Thu, 9 Nov 2023 11:25:03 -0800 Subject: [PATCH 28/39] add struct of list test for delta --- python/cudf/cudf/tests/test_parquet.py | 50 ++++++++++++++++++++++++++ 1 file changed, 50 insertions(+) diff --git a/python/cudf/cudf/tests/test_parquet.py b/python/cudf/cudf/tests/test_parquet.py index 5d8406e9c40..f164e8460de 100644 --- a/python/cudf/cudf/tests/test_parquet.py +++ b/python/cudf/cudf/tests/test_parquet.py @@ -1397,6 +1397,56 @@ def test_delta_byte_array_roundtrip(nrows, add_nulls, tmpdir): assert_eq(cdf, pcdf) +@pytest.mark.parametrize( + "nrows", [1, 2, 23, 32, 33, 34, 64, 65, 66, 128, 129, 130, 20000, 100000] +) +@pytest.mark.parametrize("add_nulls", [True, False]) +def test_delta_struct_list(tmpdir, nrows, add_nulls): + # Struct> + lists_per_row = 3 + list_size = 4 + num_rows = nrows + include_validity = add_nulls + + def list_gen_wrapped(x, y): + return list_row_gen( + int_gen, x * list_size * lists_per_row, list_size, lists_per_row + ) + + def string_list_gen_wrapped(x, y): + return list_row_gen( + string_gen, + x * list_size * lists_per_row, + list_size, + lists_per_row, + include_validity, + ) + + data = struct_gen( + [int_gen, string_gen, list_gen_wrapped, string_list_gen_wrapped], + 0, + num_rows, + include_validity, + ) + test_pdf = pa.Table.from_pydict({"sol": data}).to_pandas() + pdf_fname = tmpdir.join("test_delta_binary.parquet") + test_pdf.to_parquet( + pdf_fname, + version="2.6", + column_encoding={ + "sol.col0": "DELTA_BINARY_PACKED", + "sol.col1": "DELTA_BYTE_ARRAY", + "sol.col2.list.element.list.element": "DELTA_BINARY_PACKED", + "sol.col3.list.element.list.element": "DELTA_BYTE_ARRAY", + }, + data_page_version="2.0", + engine="pyarrow", + use_dictionary=False, + ) + cdf = cudf.read_parquet(pdf_fname) + assert_eq(cdf, cudf.from_pandas(test_pdf)) + + @pytest.mark.parametrize( "data", [ From 8b6ed5e4efb239eaf09b0629207a231a3a4ebe9d Mon Sep 17 00:00:00 2001 From: seidl Date: Thu, 9 Nov 2023 11:41:20 -0800 Subject: [PATCH 29/39] future proof test by adding param for string encoding --- python/cudf/cudf/tests/test_parquet.py | 29 +++++++++++++++----------- 1 file changed, 17 insertions(+), 12 deletions(-) diff --git a/python/cudf/cudf/tests/test_parquet.py b/python/cudf/cudf/tests/test_parquet.py index f164e8460de..c93d9ec1227 100644 --- a/python/cudf/cudf/tests/test_parquet.py +++ b/python/cudf/cudf/tests/test_parquet.py @@ -1289,6 +1289,10 @@ def test_parquet_delta_byte_array(datadir): assert_eq(cudf.read_parquet(fname), pd.read_parquet(fname)) +def delta_num_rows(): + return [1, 2, 23, 32, 33, 34, 64, 65, 66, 128, 129, 130, 20000, 50000] + + @pytest.mark.parametrize("nrows", [1, 100000]) @pytest.mark.parametrize("add_nulls", [True, False]) @pytest.mark.parametrize( @@ -1325,6 +1329,7 @@ def test_delta_binary(nrows, add_nulls, dtype, tmpdir): version="2.6", column_encoding="DELTA_BINARY_PACKED", data_page_version="2.0", + data_page_size=64 * 1024, engine="pyarrow", use_dictionary=False, ) @@ -1355,11 +1360,10 @@ def test_delta_binary(nrows, add_nulls, dtype, tmpdir): assert_eq(cdf2, cdf) -@pytest.mark.parametrize( - "nrows", [1, 2, 23, 32, 33, 34, 64, 65, 66, 128, 129, 130, 20000, 100000] -) +@pytest.mark.parametrize("nrows", delta_num_rows()) @pytest.mark.parametrize("add_nulls", [True, False]) -def test_delta_byte_array_roundtrip(nrows, add_nulls, tmpdir): +@pytest.mark.parametrize("str_encoding", ["DELTA_BYTE_ARRAY"]) +def test_delta_byte_array_roundtrip(nrows, add_nulls, str_encoding, tmpdir): null_frequency = 0.25 if add_nulls else 0 # Create a pandas dataframe with random data of mixed lengths @@ -1387,8 +1391,9 @@ def test_delta_byte_array_roundtrip(nrows, add_nulls, tmpdir): test_pdf.to_parquet( pdf_fname, version="2.6", - column_encoding="DELTA_BYTE_ARRAY", + column_encoding=str_encoding, data_page_version="2.0", + data_page_size=64 * 1024, engine="pyarrow", use_dictionary=False, ) @@ -1397,11 +1402,10 @@ def test_delta_byte_array_roundtrip(nrows, add_nulls, tmpdir): assert_eq(cdf, pcdf) -@pytest.mark.parametrize( - "nrows", [1, 2, 23, 32, 33, 34, 64, 65, 66, 128, 129, 130, 20000, 100000] -) +@pytest.mark.parametrize("nrows", delta_num_rows()) @pytest.mark.parametrize("add_nulls", [True, False]) -def test_delta_struct_list(tmpdir, nrows, add_nulls): +@pytest.mark.parametrize("str_encoding", ["DELTA_BYTE_ARRAY"]) +def test_delta_struct_list(tmpdir, nrows, add_nulls, str_encoding): # Struct> lists_per_row = 3 list_size = 4 @@ -1429,17 +1433,18 @@ def string_list_gen_wrapped(x, y): include_validity, ) test_pdf = pa.Table.from_pydict({"sol": data}).to_pandas() - pdf_fname = tmpdir.join("test_delta_binary.parquet") + pdf_fname = tmpdir.join("pdfdeltaba.parquet") test_pdf.to_parquet( pdf_fname, version="2.6", column_encoding={ "sol.col0": "DELTA_BINARY_PACKED", - "sol.col1": "DELTA_BYTE_ARRAY", + "sol.col1": str_encoding, "sol.col2.list.element.list.element": "DELTA_BINARY_PACKED", - "sol.col3.list.element.list.element": "DELTA_BYTE_ARRAY", + "sol.col3.list.element.list.element": str_encoding, }, data_page_version="2.0", + data_page_size=64 * 1024, engine="pyarrow", use_dictionary=False, ) From 5e4b2ddcec4295359a36591d05a3902502513658 Mon Sep 17 00:00:00 2001 From: Ed Seidl Date: Sun, 12 Nov 2023 17:47:33 -0800 Subject: [PATCH 30/39] fix some comments --- cpp/src/io/parquet/page_string_decode.cu | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/cpp/src/io/parquet/page_string_decode.cu b/cpp/src/io/parquet/page_string_decode.cu index af508f9ebf8..3d9a503d2c7 100644 --- a/cpp/src/io/parquet/page_string_decode.cu +++ b/cpp/src/io/parquet/page_string_decode.cu @@ -559,7 +559,7 @@ __device__ thrust::pair totalDeltaByteArraySize(uint8_t const* d * for pages with string-like data. Also calculates the number of null and valid values in the * page. Does nothing if the page mask is neither `STRING` nor `DELTA_BYTE_ARRAY`. On exit the * `num_nulls`, `num_valids`, `start_val` and `end_val` fields of the `PageInfo` struct will be - * populated. Also fills in the `temp_string_size` field if rows are to be skipped. + * populated. * * @param pages All pages to be decoded * @param chunks All chunks to be decoded @@ -622,7 +622,8 @@ __global__ void __launch_bounds__(preprocess_block_size) gpuComputeStringPageBou * @brief Kernel for computing string page output size information for delta_byte_array encoding. * * This call ignores columns that are not DELTA_BYTE_ARRAY encoded. On exit the `str_bytes` field - * of the `PageInfo` struct will be populated. + * of the `PageInfo` struct will be populated. Also fills in the `temp_string_size` field if rows + * are to be skipped. * * @param pages All pages to be decoded * @param chunks All chunks to be decoded @@ -658,7 +659,7 @@ __global__ void __launch_bounds__(delta_preproc_block_size) gpuComputeDeltaPageS // just need to parse the header of the first delta binary block to get values_per_mb delta_binary_decoder db; db.init_binary_block(s->data_start, s->data_end); - // save enough for one mimi-block plus some extra to save the last_string + // save enough for one mini-block plus some extra to save the last_string pp->temp_string_size = s->dtype_len_in * (db.values_per_mb + 1); } } From 8338207b93cd8c1b851a2fdbc2a4097899fb3270 Mon Sep 17 00:00:00 2001 From: seidl Date: Mon, 13 Nov 2023 08:55:43 -0800 Subject: [PATCH 31/39] a few fixes suggested in review --- cpp/src/io/parquet/page_data.cu | 10 +++++++--- cpp/src/io/parquet/page_string_decode.cu | 12 ++++++------ 2 files changed, 13 insertions(+), 9 deletions(-) diff --git a/cpp/src/io/parquet/page_data.cu b/cpp/src/io/parquet/page_data.cu index 1f3ab0040e7..e95f6f48a10 100644 --- a/cpp/src/io/parquet/page_data.cu +++ b/cpp/src/io/parquet/page_data.cu @@ -449,9 +449,13 @@ __global__ void __launch_bounds__(decode_block_size) int out_thread0; [[maybe_unused]] null_count_back_copier _{s, t}; - auto const mask = decode_kernel_mask::GENERAL; - if (!setupLocalPageInfo( - s, &pages[page_idx], chunks, min_row, num_rows, mask_filter{mask}, true)) { + if (!setupLocalPageInfo(s, + &pages[page_idx], + chunks, + min_row, + num_rows, + mask_filter{decode_kernel_mask::GENERAL}, + true)) { return; } diff --git a/cpp/src/io/parquet/page_string_decode.cu b/cpp/src/io/parquet/page_string_decode.cu index 3d9a503d2c7..2cbdf493b54 100644 --- a/cpp/src/io/parquet/page_string_decode.cu +++ b/cpp/src/io/parquet/page_string_decode.cu @@ -1018,12 +1018,12 @@ void ComputePageStringSizes(cudf::detail::hostdevice_vector& pages, if (need_sizes) { // sum up all of the temp_string_sizes auto const page_sizes = [] __device__(PageInfo const& page) { return page.temp_string_size; }; - int64_t total_size = thrust::transform_reduce(rmm::exec_policy(stream), - pages.d_begin(), - pages.d_end(), - page_sizes, - 0L, - thrust::plus{}); + auto const total_size = thrust::transform_reduce(rmm::exec_policy(stream), + pages.d_begin(), + pages.d_end(), + page_sizes, + 0L, + thrust::plus{}); // now do an exclusive scan over the temp_string_sizes to get offsets for each // page's chunk of the temp buffer From 8bf6edc2330275eb7f4a17f0789a6b1e06ee51ad Mon Sep 17 00:00:00 2001 From: seidl Date: Tue, 14 Nov 2023 09:25:39 -0800 Subject: [PATCH 32/39] add sanity check to failing test --- python/cudf/cudf/tests/test_parquet.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/python/cudf/cudf/tests/test_parquet.py b/python/cudf/cudf/tests/test_parquet.py index c93d9ec1227..af4d0294293 100644 --- a/python/cudf/cudf/tests/test_parquet.py +++ b/python/cudf/cudf/tests/test_parquet.py @@ -1448,6 +1448,8 @@ def string_list_gen_wrapped(x, y): engine="pyarrow", use_dictionary=False, ) + # sanity check to verify file is written properly + assert_eq(test_pdf, pd.read_parquet(pdf_fname)) cdf = cudf.read_parquet(pdf_fname) assert_eq(cdf, cudf.from_pandas(test_pdf)) From ea4c283077c46a6f2fca09fe7fe46cc26e11a93f Mon Sep 17 00:00:00 2001 From: seidl Date: Tue, 14 Nov 2023 14:28:14 -0800 Subject: [PATCH 33/39] fix overrun caused by large mini-blocks --- cpp/src/io/parquet/delta_binary.cuh | 25 +++++++++++----- cpp/src/io/parquet/page_delta_decode.cu | 40 ++++++++----------------- 2 files changed, 30 insertions(+), 35 deletions(-) diff --git a/cpp/src/io/parquet/delta_binary.cuh b/cpp/src/io/parquet/delta_binary.cuh index ecaf0eb4fbe..db44e2cd662 100644 --- a/cpp/src/io/parquet/delta_binary.cuh +++ b/cpp/src/io/parquet/delta_binary.cuh @@ -40,7 +40,10 @@ namespace cudf::io::parquet::detail { // block to ensure that all encoded values are positive. The deltas for each mini-block are bit // packed using the same encoding as the RLE/Bit-Packing Hybrid encoder. -// we decode one mini-block at a time. max mini-block size seen is 64. +// The first pass decodes `values_per_mb` values, and then the second pass does another +// batch of size `values_per_mb`. The largest value for values_per_miniblock among the +// major writers seems to be 64, so 2 * 64 should be good. We save the first value separately +// since it is not encoded in the first mini-block. constexpr int delta_rolling_buf_size = 128; /** @@ -84,7 +87,8 @@ struct delta_binary_decoder { uleb128_t mini_block_count; // usually 4, chosen such that block_size/mini_block_count is a // multiple of 32 uleb128_t value_count; // total values encoded in the block - zigzag128_t last_value; // last value decoded, initialized to first_value from header + zigzag128_t first_value; // initial value, stored in the header + zigzag128_t last_value; // last value decoded uint32_t values_per_mb; // block_size / mini_block_count, must be multiple of 32 uint32_t current_value_idx; // current value index, initialized to 0 at start of block @@ -96,6 +100,13 @@ struct delta_binary_decoder { uleb128_t value[delta_rolling_buf_size]; // circular buffer of delta values + // returns the value stored in the `value` array at index + // `rolling_index(idx)`. If `idx` is `0`, then return `first_value`. + constexpr zigzag128_t value_at(size_type idx) + { + return idx == 0 ? first_value : value[rolling_index(idx)]; + } + // returns the number of values encoded in the block data. when all_values is true, // account for the first value in the header. otherwise just count the values encoded // in the mini-block data. @@ -139,7 +150,8 @@ struct delta_binary_decoder { block_size = get_uleb128(d_start, d_end); mini_block_count = get_uleb128(d_start, d_end); value_count = get_uleb128(d_start, d_end); - last_value = get_zz128(d_start, d_end); + first_value = get_zz128(d_start, d_end); + last_value = first_value; current_value_idx = 0; values_per_mb = block_size / mini_block_count; @@ -202,12 +214,9 @@ struct delta_binary_decoder { using cudf::detail::warp_size; if (current_value_idx >= value_count) { return; } - // need to save first value from header on first pass + // need to account for the first value from header on first pass if (current_value_idx == 0) { - if (lane_id == 0) { - current_value_idx++; - value[0] = last_value; - } + if (lane_id == 0) { current_value_idx++; } __syncwarp(); if (current_value_idx >= value_count) { return; } } diff --git a/cpp/src/io/parquet/page_delta_decode.cu b/cpp/src/io/parquet/page_delta_decode.cu index 5a4ef36e071..9e3b3abda87 100644 --- a/cpp/src/io/parquet/page_delta_decode.cu +++ b/cpp/src/io/parquet/page_delta_decode.cu @@ -82,8 +82,7 @@ struct delta_byte_array_decoder { __shared__ __align__(8) uint8_t const* offsets[warp_size]; uint32_t const ln_idx = start_idx + lane_id; - uint32_t const src_idx = rolling_index(ln_idx); - uint64_t prefix_len = ln_idx < end_idx ? prefixes.value[src_idx] : 0; + uint64_t prefix_len = ln_idx < end_idx ? prefixes.value_at(ln_idx) : 0; uint8_t* const lane_out = ln_idx < end_idx ? strings_out + offset : nullptr; prefix_lens[lane_id] = prefix_len; @@ -143,13 +142,12 @@ struct delta_byte_array_decoder { uint32_t const ln_idx = idx + lane_id; // calculate offsets into suffix data - uint32_t const src_idx = rolling_index(ln_idx); - uint64_t const suffix_len = ln_idx < end ? suffixes.value[src_idx] : 0; + uint64_t const suffix_len = ln_idx < end ? suffixes.value_at(ln_idx) : 0; uint64_t suffix_off = 0; WarpScan(scan_temp).ExclusiveSum(suffix_len, suffix_off); // calculate offsets into string data - uint64_t const prefix_len = ln_idx < end ? prefixes.value[src_idx] : 0; + uint64_t const prefix_len = ln_idx < end ? prefixes.value_at(ln_idx) : 0; uint64_t const string_len = prefix_len + suffix_len; // get offset into output for each lane @@ -224,9 +222,8 @@ struct delta_byte_array_decoder { uint64_t string_total = 0; for (int idx = start_idx; idx < end_idx; idx++) { - uint32_t const src_idx = rolling_index(idx); - uint64_t const suffix_len = suffixes.value[src_idx]; - uint64_t const prefix_len = prefixes.value[src_idx]; + uint64_t const suffix_len = suffixes.value_at(idx); + uint64_t const prefix_len = prefixes.value_at(idx); uint64_t const string_len = prefix_len + suffix_len; // copy prefix and suffix data into current strings_out position @@ -395,23 +392,12 @@ __global__ void __launch_bounds__(96) // place value for this thread if (dst_pos >= 0 && sp < target_pos) { void* const dst = nesting_info_base[leaf_level_index].data_out + dst_pos * s->dtype_len; + auto const val = db->value_at(sp + skipped_leaf_values); switch (s->dtype_len) { - case 1: - *static_cast(dst) = - db->value[rolling_index(sp + skipped_leaf_values)]; - break; - case 2: - *static_cast(dst) = - db->value[rolling_index(sp + skipped_leaf_values)]; - break; - case 4: - *static_cast(dst) = - db->value[rolling_index(sp + skipped_leaf_values)]; - break; - case 8: - *static_cast(dst) = - db->value[rolling_index(sp + skipped_leaf_values)]; - break; + case 1: *static_cast(dst) = val; break; + case 2: *static_cast(dst) = val; break; + case 4: *static_cast(dst) = val; break; + case 8: *static_cast(dst) = val; break; } } } @@ -505,7 +491,7 @@ __global__ void __launch_bounds__(decode_block_size) uint32_t const src_pos = s->src_pos; if (t < 3 * warp_size) { // warp 0..2 - target_pos = min(src_pos + 2 * (batch_size), s->nz_count + s->first_row + batch_size); + target_pos = min(src_pos + 2 * batch_size, s->nz_count + s->first_row + batch_size); } else { // warp 3 target_pos = min(s->nz_count, src_pos + batch_size); } @@ -549,8 +535,8 @@ __global__ void __launch_bounds__(decode_block_size) if (dst_pos >= 0 && sp < target_pos) { auto const offptr = reinterpret_cast(nesting_info_base[leaf_level_index].data_out) + dst_pos; - auto const src_idx = rolling_index(sp + skipped_leaf_values); - *offptr = prefix_db->value[src_idx] + suffix_db->value[src_idx]; + auto const src_idx = sp + skipped_leaf_values; + *offptr = prefix_db->value_at(src_idx) + suffix_db->value_at(src_idx); } __syncwarp(); } From cf5abe58c6b4fcf6c92c0645d07c6c2c843385a2 Mon Sep 17 00:00:00 2001 From: seidl Date: Tue, 14 Nov 2023 15:03:35 -0800 Subject: [PATCH 34/39] check that the mini-block size is not too large to handle --- cpp/src/io/parquet/delta_binary.cuh | 5 ++++- cpp/src/io/parquet/page_delta_decode.cu | 8 ++++++++ cpp/src/io/parquet/parquet_gpu.hpp | 17 +++++++++-------- 3 files changed, 21 insertions(+), 9 deletions(-) diff --git a/cpp/src/io/parquet/delta_binary.cuh b/cpp/src/io/parquet/delta_binary.cuh index db44e2cd662..ccc28791071 100644 --- a/cpp/src/io/parquet/delta_binary.cuh +++ b/cpp/src/io/parquet/delta_binary.cuh @@ -40,11 +40,14 @@ namespace cudf::io::parquet::detail { // block to ensure that all encoded values are positive. The deltas for each mini-block are bit // packed using the same encoding as the RLE/Bit-Packing Hybrid encoder. +// The largest mini-block size we can currently support. +constexpr int max_delta_mini_block_size = 64; + // The first pass decodes `values_per_mb` values, and then the second pass does another // batch of size `values_per_mb`. The largest value for values_per_miniblock among the // major writers seems to be 64, so 2 * 64 should be good. We save the first value separately // since it is not encoded in the first mini-block. -constexpr int delta_rolling_buf_size = 128; +constexpr int delta_rolling_buf_size = 2 * max_delta_mini_block_size; /** * @brief Read a ULEB128 varint integer diff --git a/cpp/src/io/parquet/page_delta_decode.cu b/cpp/src/io/parquet/page_delta_decode.cu index 9e3b3abda87..b0e9fa3a4ab 100644 --- a/cpp/src/io/parquet/page_delta_decode.cu +++ b/cpp/src/io/parquet/page_delta_decode.cu @@ -345,6 +345,10 @@ __global__ void __launch_bounds__(96) __syncthreads(); auto const batch_size = db->values_per_mb; + if (batch_size > max_delta_mini_block_size) { + set_error(static_cast(decode_error::DELTA_PARAMS_UNSUPPORTED), error_code); + return; + } // if skipped_leaf_values is non-zero, then we need to decode up to the first mini-block // that has a value we need. @@ -479,6 +483,10 @@ __global__ void __launch_bounds__(decode_block_size) auto strings_data = nesting_info_base[leaf_level_index].string_out; auto const batch_size = prefix_db->values_per_mb; + if (batch_size > max_delta_mini_block_size) { + set_error(static_cast(decode_error::DELTA_PARAMS_UNSUPPORTED), error_code); + return; + } // if this is a bounds page and nested, then we need to skip up front. non-nested will work // its way through the page. diff --git a/cpp/src/io/parquet/parquet_gpu.hpp b/cpp/src/io/parquet/parquet_gpu.hpp index d36cf646095..129d4e4d28c 100644 --- a/cpp/src/io/parquet/parquet_gpu.hpp +++ b/cpp/src/io/parquet/parquet_gpu.hpp @@ -88,14 +88,15 @@ constexpr void set_error(int32_t error, int32_t* error_code) * These values are used as bitmasks, so they must be powers of 2. */ enum class decode_error : int32_t { - DATA_STREAM_OVERRUN = 0x1, - LEVEL_STREAM_OVERRUN = 0x2, - UNSUPPORTED_ENCODING = 0x4, - INVALID_LEVEL_RUN = 0x8, - INVALID_DATA_TYPE = 0x10, - EMPTY_PAGE = 0x20, - INVALID_DICT_WIDTH = 0x40, - DELTA_PARAM_MISMATCH = 0x80, + DATA_STREAM_OVERRUN = 0x1, + LEVEL_STREAM_OVERRUN = 0x2, + UNSUPPORTED_ENCODING = 0x4, + INVALID_LEVEL_RUN = 0x8, + INVALID_DATA_TYPE = 0x10, + EMPTY_PAGE = 0x20, + INVALID_DICT_WIDTH = 0x40, + DELTA_PARAM_MISMATCH = 0x80, + DELTA_PARAMS_UNSUPPORTED = 0x100, }; /** From b697cb82e40a04cc7643caab8b9ba2ca097e8d74 Mon Sep 17 00:00:00 2001 From: Ed Seidl Date: Wed, 15 Nov 2023 09:31:36 -0800 Subject: [PATCH 35/39] smack redundant statement Co-authored-by: nvdbaranec <56695930+nvdbaranec@users.noreply.github.com> --- cpp/src/io/parquet/page_string_decode.cu | 1 - 1 file changed, 1 deletion(-) diff --git a/cpp/src/io/parquet/page_string_decode.cu b/cpp/src/io/parquet/page_string_decode.cu index 2cbdf493b54..60bdbaffbe8 100644 --- a/cpp/src/io/parquet/page_string_decode.cu +++ b/cpp/src/io/parquet/page_string_decode.cu @@ -528,7 +528,6 @@ __device__ thrust::pair totalDeltaByteArraySize(uint8_t const* d // get sum for warp. // note: warp_sum will only be valid on lane 0. - using cudf::detail::warp_size; auto const warp_sum = WarpReduce(temp_storage[warp_id]).Sum(lane_sum); auto const warp_max = WarpReduce(temp_storage[warp_id]).Reduce(lane_max, cub::Max()); From 07b2f792d7092b25d64d6fc4697c6c57b879ce53 Mon Sep 17 00:00:00 2001 From: Ed Seidl Date: Wed, 15 Nov 2023 09:32:15 -0800 Subject: [PATCH 36/39] add const Co-authored-by: nvdbaranec <56695930+nvdbaranec@users.noreply.github.com> --- cpp/src/io/parquet/page_string_decode.cu | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cpp/src/io/parquet/page_string_decode.cu b/cpp/src/io/parquet/page_string_decode.cu index 60bdbaffbe8..ef637141732 100644 --- a/cpp/src/io/parquet/page_string_decode.cu +++ b/cpp/src/io/parquet/page_string_decode.cu @@ -544,7 +544,7 @@ __device__ thrust::pair totalDeltaByteArraySize(uint8_t const* d // Sum up prefix and suffix max lengths to get a max possible string length. Multiply that // by the number of strings in a mini-block, plus one to save the last string. - auto temp_bytes = + auto const temp_bytes = cudf::detail::single_lane_block_sum_reduce(max_len) * (db->values_per_mb + 1); From 883a236d5a274345cb477277aa2290dc128d8ad9 Mon Sep 17 00:00:00 2001 From: seidl Date: Wed, 15 Nov 2023 10:13:18 -0800 Subject: [PATCH 37/39] remove some unnecessary syncthreads calls --- cpp/src/io/parquet/page_data.cu | 2 +- cpp/src/io/parquet/page_delta_decode.cu | 2 -- cpp/src/io/parquet/page_string_decode.cu | 2 +- 3 files changed, 2 insertions(+), 4 deletions(-) diff --git a/cpp/src/io/parquet/page_data.cu b/cpp/src/io/parquet/page_data.cu index e95f6f48a10..100de2653ff 100644 --- a/cpp/src/io/parquet/page_data.cu +++ b/cpp/src/io/parquet/page_data.cu @@ -491,7 +491,7 @@ __global__ void __launch_bounds__(decode_block_size) target_pos = min(s->nz_count, src_pos + decode_block_size - out_thread0); if (out_thread0 > 32) { target_pos = min(target_pos, s->dict_pos); } } - __syncthreads(); + if (t < 32) { // decode repetition and definition levels. // - update validity vectors diff --git a/cpp/src/io/parquet/page_delta_decode.cu b/cpp/src/io/parquet/page_delta_decode.cu index b0e9fa3a4ab..4e0eeab0ba9 100644 --- a/cpp/src/io/parquet/page_delta_decode.cu +++ b/cpp/src/io/parquet/page_delta_decode.cu @@ -364,7 +364,6 @@ __global__ void __launch_bounds__(96) } else { // warp2 target_pos = min(s->nz_count, src_pos + batch_size); } - __syncthreads(); // warp0 will decode the rep/def levels, warp1 will unpack a mini-batch of deltas. // warp2 waits one cycle for warps 0/1 to produce a batch, and then stuffs values @@ -503,7 +502,6 @@ __global__ void __launch_bounds__(decode_block_size) } else { // warp 3 target_pos = min(s->nz_count, src_pos + batch_size); } - __syncthreads(); // warp0 will decode the rep/def levels, warp1 will unpack a mini-batch of prefixes, warp 2 will // unpack a mini-batch of suffixes. warp3 waits one cycle for warps 0-2 to produce a batch, and diff --git a/cpp/src/io/parquet/page_string_decode.cu b/cpp/src/io/parquet/page_string_decode.cu index ef637141732..be65612acbe 100644 --- a/cpp/src/io/parquet/page_string_decode.cu +++ b/cpp/src/io/parquet/page_string_decode.cu @@ -833,7 +833,7 @@ __global__ void __launch_bounds__(decode_block_size) target_pos = min(s->nz_count, src_pos + decode_block_size - out_thread0); if (out_thread0 > 32) { target_pos = min(target_pos, s->dict_pos); } } - __syncthreads(); + if (t < 32) { // decode repetition and definition levels. // - update validity vectors From 0844d8e249cc4b4036122976527b00632c25bfed Mon Sep 17 00:00:00 2001 From: seidl Date: Thu, 16 Nov 2023 11:49:06 -0800 Subject: [PATCH 38/39] Revert "remove some unnecessary syncthreads calls" This reverts commit 883a236d5a274345cb477277aa2290dc128d8ad9. --- cpp/src/io/parquet/page_data.cu | 2 +- cpp/src/io/parquet/page_delta_decode.cu | 2 ++ cpp/src/io/parquet/page_string_decode.cu | 2 +- 3 files changed, 4 insertions(+), 2 deletions(-) diff --git a/cpp/src/io/parquet/page_data.cu b/cpp/src/io/parquet/page_data.cu index 100de2653ff..e95f6f48a10 100644 --- a/cpp/src/io/parquet/page_data.cu +++ b/cpp/src/io/parquet/page_data.cu @@ -491,7 +491,7 @@ __global__ void __launch_bounds__(decode_block_size) target_pos = min(s->nz_count, src_pos + decode_block_size - out_thread0); if (out_thread0 > 32) { target_pos = min(target_pos, s->dict_pos); } } - + __syncthreads(); if (t < 32) { // decode repetition and definition levels. // - update validity vectors diff --git a/cpp/src/io/parquet/page_delta_decode.cu b/cpp/src/io/parquet/page_delta_decode.cu index 4e0eeab0ba9..b0e9fa3a4ab 100644 --- a/cpp/src/io/parquet/page_delta_decode.cu +++ b/cpp/src/io/parquet/page_delta_decode.cu @@ -364,6 +364,7 @@ __global__ void __launch_bounds__(96) } else { // warp2 target_pos = min(s->nz_count, src_pos + batch_size); } + __syncthreads(); // warp0 will decode the rep/def levels, warp1 will unpack a mini-batch of deltas. // warp2 waits one cycle for warps 0/1 to produce a batch, and then stuffs values @@ -502,6 +503,7 @@ __global__ void __launch_bounds__(decode_block_size) } else { // warp 3 target_pos = min(s->nz_count, src_pos + batch_size); } + __syncthreads(); // warp0 will decode the rep/def levels, warp1 will unpack a mini-batch of prefixes, warp 2 will // unpack a mini-batch of suffixes. warp3 waits one cycle for warps 0-2 to produce a batch, and diff --git a/cpp/src/io/parquet/page_string_decode.cu b/cpp/src/io/parquet/page_string_decode.cu index be65612acbe..ef637141732 100644 --- a/cpp/src/io/parquet/page_string_decode.cu +++ b/cpp/src/io/parquet/page_string_decode.cu @@ -833,7 +833,7 @@ __global__ void __launch_bounds__(decode_block_size) target_pos = min(s->nz_count, src_pos + decode_block_size - out_thread0); if (out_thread0 > 32) { target_pos = min(target_pos, s->dict_pos); } } - + __syncthreads(); if (t < 32) { // decode repetition and definition levels. // - update validity vectors From 2f83f8a4b37a9f3f1fd0c5d37d893e150e16ecab Mon Sep 17 00:00:00 2001 From: seidl Date: Thu, 16 Nov 2023 11:52:05 -0800 Subject: [PATCH 39/39] add a TODO to remove some syncthreads calls --- cpp/src/io/parquet/page_data.cu | 1 + cpp/src/io/parquet/page_delta_decode.cu | 2 ++ cpp/src/io/parquet/page_string_decode.cu | 1 + 3 files changed, 4 insertions(+) diff --git a/cpp/src/io/parquet/page_data.cu b/cpp/src/io/parquet/page_data.cu index e95f6f48a10..0c53877f7c7 100644 --- a/cpp/src/io/parquet/page_data.cu +++ b/cpp/src/io/parquet/page_data.cu @@ -491,6 +491,7 @@ __global__ void __launch_bounds__(decode_block_size) target_pos = min(s->nz_count, src_pos + decode_block_size - out_thread0); if (out_thread0 > 32) { target_pos = min(target_pos, s->dict_pos); } } + // TODO(ets): see if this sync can be removed __syncthreads(); if (t < 32) { // decode repetition and definition levels. diff --git a/cpp/src/io/parquet/page_delta_decode.cu b/cpp/src/io/parquet/page_delta_decode.cu index b0e9fa3a4ab..bc025c6fc3e 100644 --- a/cpp/src/io/parquet/page_delta_decode.cu +++ b/cpp/src/io/parquet/page_delta_decode.cu @@ -364,6 +364,7 @@ __global__ void __launch_bounds__(96) } else { // warp2 target_pos = min(s->nz_count, src_pos + batch_size); } + // TODO(ets): see if this sync can be removed __syncthreads(); // warp0 will decode the rep/def levels, warp1 will unpack a mini-batch of deltas. @@ -503,6 +504,7 @@ __global__ void __launch_bounds__(decode_block_size) } else { // warp 3 target_pos = min(s->nz_count, src_pos + batch_size); } + // TODO(ets): see if this sync can be removed __syncthreads(); // warp0 will decode the rep/def levels, warp1 will unpack a mini-batch of prefixes, warp 2 will diff --git a/cpp/src/io/parquet/page_string_decode.cu b/cpp/src/io/parquet/page_string_decode.cu index ef637141732..e29db042401 100644 --- a/cpp/src/io/parquet/page_string_decode.cu +++ b/cpp/src/io/parquet/page_string_decode.cu @@ -833,6 +833,7 @@ __global__ void __launch_bounds__(decode_block_size) target_pos = min(s->nz_count, src_pos + decode_block_size - out_thread0); if (out_thread0 > 32) { target_pos = min(target_pos, s->dict_pos); } } + // TODO(ets): see if this sync can be removed __syncthreads(); if (t < 32) { // decode repetition and definition levels.