From 7ca6570a7007f801e8b1699500d266fbb16404e7 Mon Sep 17 00:00:00 2001 From: Devavret Makkar Date: Sat, 23 Oct 2021 03:26:33 +0530 Subject: [PATCH 01/71] First working version of partitioned write --- cpp/src/io/parquet/chunk_dict.cu | 6 + cpp/src/io/parquet/page_enc.cu | 37 ++- cpp/src/io/parquet/parquet_gpu.hpp | 17 +- cpp/src/io/parquet/writer_impl.cu | 443 +++++++++++++++++++---------- cpp/src/io/parquet/writer_impl.hpp | 11 +- cpp/tests/io/parquet_test.cpp | 17 +- 6 files changed, 359 insertions(+), 172 deletions(-) diff --git a/cpp/src/io/parquet/chunk_dict.cu b/cpp/src/io/parquet/chunk_dict.cu index 64b3dd69c0d..e6a9041d600 100644 --- a/cpp/src/io/parquet/chunk_dict.cu +++ b/cpp/src/io/parquet/chunk_dict.cu @@ -335,6 +335,7 @@ void initialize_chunk_hash_maps(device_span chunks, rmm::cuda_st } void populate_chunk_hash_maps(cudf::detail::device_2dspan chunks, + cudf::detail::device_2dspan frags, size_type num_rows, rmm::cuda_stream_view stream) { @@ -343,6 +344,11 @@ void populate_chunk_hash_maps(cudf::detail::device_2dspan chunks auto const num_columns = chunks.size().second; dim3 const dim_grid(grid_x.num_blocks, num_columns); + // Convert to a per-fragment kernel. It is like that already and I know we cannot avoid fragments + // anymore. The only other alternative is using row_bit_count to find per-row size and then use it + // to calculate rowgroup boundaries. And that one doesn't exclude null data size. Maybe in the + // future we can remove fragments and allow rowgroups to have less than 5000 rows but it's not + // important right now populate_chunk_hash_maps_kernel <<>>(chunks, num_rows); } diff --git a/cpp/src/io/parquet/page_enc.cu b/cpp/src/io/parquet/page_enc.cu index 48490426db7..810dd1a45c0 100644 --- a/cpp/src/io/parquet/page_enc.cu +++ b/cpp/src/io/parquet/page_enc.cu @@ -24,6 +24,7 @@ #include +#include #include #include #include @@ -48,6 +49,7 @@ constexpr uint32_t rle_buffer_size = (1 << 9); struct frag_init_state_s { parquet_column_device_view col; PageFragment frag; + // TODO: replace this with frag.start_value size_type start_value_idx; }; @@ -112,8 +114,10 @@ template __global__ void __launch_bounds__(block_size) gpuInitPageFragments(device_2dspan frag, device_span col_desc, + device_span partitions, + device_span part_frag_offset, uint32_t fragment_size, - uint32_t max_num_rows) + uint32_t max_num_rows) // TODO: remove { __shared__ __align__(16) frag_init_state_s state_g; @@ -122,15 +126,22 @@ __global__ void __launch_bounds__(block_size) frag_init_state_s* const s = &state_g; uint32_t t = threadIdx.x; - uint32_t start_row, dtype_len, dtype; + int frag_y = blockIdx.y; + uint32_t dtype_len, dtype; if (t == 0) s->col = col_desc[blockIdx.x]; __syncthreads(); - start_row = blockIdx.y * fragment_size; if (!t) { - // frag.num_rows = fragment_size except for the last page fragment which can be smaller. + // Find which partition this fragment came from + auto it = + thrust::upper_bound(thrust::seq, part_frag_offset.begin(), part_frag_offset.end(), frag_y); + int p = it - part_frag_offset.begin() - 1; + int part_end_row = partitions[p].start_row + partitions[p].num_rows; + s->frag.start_row = (frag_y - part_frag_offset[p]) * fragment_size + partitions[p].start_row; + + // frag.num_rows = fragment_size except for the last fragment in partition which can be smaller. // num_rows is fixed but fragment size could be larger if the data is strings or nested. - s->frag.num_rows = min(fragment_size, max_num_rows - min(start_row, max_num_rows)); + s->frag.num_rows = min(fragment_size, part_end_row - s->frag.start_row); s->frag.num_dict_vals = 0; s->frag.fragment_data_size = 0; s->frag.dict_data_size = 0; @@ -140,12 +151,12 @@ __global__ void __launch_bounds__(block_size) // off_11 = off[i], off_12 = off[i+50] // off_21 = child.off[off_11], off_22 = child.off[off_12] // etc... - size_type end_value_idx = start_row + s->frag.num_rows; + size_type end_value_idx = s->frag.start_row + s->frag.num_rows; if (s->col.parent_column == nullptr) { - s->start_value_idx = start_row; + s->start_value_idx = s->frag.start_row; } else { auto col = *(s->col.parent_column); - auto current_start_value_idx = start_row; + auto current_start_value_idx = s->frag.start_row; while (col.type().id() == type_id::LIST or col.type().id() == type_id::STRUCT) { if (col.type().id() == type_id::STRUCT) { current_start_value_idx += col.offset(); @@ -168,8 +179,8 @@ __global__ void __launch_bounds__(block_size) // For nested schemas, the number of values in a fragment is not directly related to the // number of encoded data elements or the number of rows. It is simply the number of // repetition/definition values which together encode validity and nesting information. - size_type first_level_val_idx = s->col.level_offsets[start_row]; - size_type last_level_val_idx = s->col.level_offsets[start_row + s->frag.num_rows]; + size_type first_level_val_idx = s->col.level_offsets[s->frag.start_row]; + size_type last_level_val_idx = s->col.level_offsets[s->frag.start_row + s->frag.num_rows]; s->frag.num_values = last_level_val_idx - first_level_val_idx; } else { s->frag.num_values = s->frag.num_rows; @@ -1944,6 +1955,8 @@ dremel_data get_dremel_data(column_view h_col, */ void InitPageFragments(device_2dspan frag, device_span col_desc, + device_span partitions, + device_span part_frag_offset, uint32_t fragment_size, uint32_t num_rows, rmm::cuda_stream_view stream) @@ -1951,8 +1964,8 @@ void InitPageFragments(device_2dspan frag, auto num_columns = frag.size().first; auto num_fragments_per_column = frag.size().second; dim3 dim_grid(num_columns, num_fragments_per_column); // 1 threadblock per fragment - gpuInitPageFragments<512> - <<>>(frag, col_desc, fragment_size, num_rows); + gpuInitPageFragments<512><<>>( + frag, col_desc, partitions, part_frag_offset, fragment_size, num_rows); } /** diff --git a/cpp/src/io/parquet/parquet_gpu.hpp b/cpp/src/io/parquet/parquet_gpu.hpp index ac2e6ba5cfb..ba37ff8be78 100644 --- a/cpp/src/io/parquet/parquet_gpu.hpp +++ b/cpp/src/io/parquet/parquet_gpu.hpp @@ -250,6 +250,11 @@ struct parquet_column_device_view : stats_column_desc { //!< col.nullable() in case of chunked writing. }; +struct partition_info { + size_type start_row; + size_type num_rows; +}; + constexpr int max_page_fragment_size = 5000; //!< Max number of rows in a page fragment /** @@ -262,8 +267,11 @@ struct PageFragment { uint32_t start_value_idx; uint32_t num_leaf_values; //!< Number of leaf values in fragment. Does not include nulls at //!< non-leaf level - uint16_t num_rows; //!< Number of rows in fragment - uint16_t num_dict_vals; //!< Number of unique dictionary entries + + // Add a start_row member because fragments no longer 5000 rows each + uint16_t start_row; //!< First row in fragment + uint16_t num_rows; //!< Number of rows in fragment + uint16_t num_dict_vals; //!< Number of unique dictionary entries }; /// Size of hash used for building dictionaries @@ -301,6 +309,8 @@ struct EncPage; struct EncColumnChunk { parquet_column_device_view const* col_desc; //!< Column description size_type col_desc_id; + + // Add a num fragments PageFragment* fragments; //!< First fragment in chunk uint8_t* uncompressed_bfr; //!< Uncompressed page data uint8_t* compressed_bfr; //!< Compressed page data @@ -469,6 +479,8 @@ dremel_data get_dremel_data(column_view h_col, */ void InitPageFragments(cudf::detail::device_2dspan frag, device_span col_desc, + device_span partitions, + device_span first_frag_in_part, uint32_t fragment_size, uint32_t num_rows, rmm::cuda_stream_view stream); @@ -502,6 +514,7 @@ void initialize_chunk_hash_maps(device_span chunks, rmm::cuda_st * @param stream CUDA stream to use */ void populate_chunk_hash_maps(cudf::detail::device_2dspan chunks, + cudf::detail::device_2dspan frags, size_type num_rows, rmm::cuda_stream_view stream); diff --git a/cpp/src/io/parquet/writer_impl.cu b/cpp/src/io/parquet/writer_impl.cu index 5b60a81e09b..458dd0e35e1 100644 --- a/cpp/src/io/parquet/writer_impl.cu +++ b/cpp/src/io/parquet/writer_impl.cu @@ -26,6 +26,7 @@ #include "compact_protocol_writer.hpp" #include +#include #include #include #include @@ -39,6 +40,8 @@ #include +#include + #include #include #include @@ -75,6 +78,35 @@ parquet::Compression to_parquet_compression(compression_type compression) } // namespace +// TODO: now I think partition is a better name than file +struct per_file_metadata { + per_file_metadata(int num_files) : files(num_files) {} + + FileMetaData get_metadata(int part) + { + FileMetaData meta{}; + meta.version = this->version; + meta.schema = this->schema; + meta.num_rows = this->files[part].num_rows; + meta.row_groups = this->files[part].row_groups; + meta.key_value_metadata = this->files[part].key_value_metadata; + meta.created_by = this->created_by; + meta.column_order_listsize = this->column_order_listsize; + return meta; + } + + int32_t version = 0; + std::vector schema; + struct per_file_members { + int64_t num_rows = 0; + std::vector row_groups; + std::vector key_value_metadata; + }; + std::vector files; + std::string created_by = ""; + uint32_t column_order_listsize = 0; +}; + struct linked_column_view; using LinkedColPtr = std::shared_ptr; @@ -736,10 +768,24 @@ gpu::parquet_column_device_view parquet_column_view::get_device_view( void writer::impl::init_page_fragments(cudf::detail::hostdevice_2dvector& frag, device_span col_desc, + std::vector> partitions, + device_span part_frag_offset, uint32_t num_rows, uint32_t fragment_size) { - gpu::InitPageFragments(frag, col_desc, fragment_size, num_rows, stream); + // TODO: partitions as pairs aren't convertible to device_uvector because device_uvector does not + // allow non-trivially-copyable types and std::pair has a copy constructor. Find a way to expose + // gpu::partition_info in public header and accept the partitions as that. OR at least make this + // conversion early on in the write() call so that we don't have to do part.first and part.second + // everywhere + std::vector h_partitions; + std::transform( + partitions.begin(), partitions.end(), std::back_inserter(h_partitions), [](auto const& part) { + return gpu::partition_info{part.first, part.second}; + }); + auto d_partitions = cudf::detail::make_device_uvector_async(h_partitions, stream); + gpu::InitPageFragments( + frag, col_desc, d_partitions, part_frag_offset, fragment_size, num_rows, stream); frag.device_to_host(stream, true); } @@ -771,6 +817,7 @@ void writer::impl::init_page_sizes(hostdevice_2dvector& chu auto build_chunk_dictionaries(hostdevice_2dvector& chunks, host_span col_desc, + device_2dspan frags, uint32_t num_rows, rmm::cuda_stream_view stream) { @@ -801,7 +848,7 @@ auto build_chunk_dictionaries(hostdevice_2dvector& chunks, chunks.host_to_device(stream); gpu::initialize_chunk_hash_maps(chunks.device_view().flat_view(), stream); - gpu::populate_chunk_hash_maps(chunks, num_rows, stream); + gpu::populate_chunk_hash_maps(chunks, frags, num_rows, stream); chunks.device_to_host(stream, true); @@ -1028,9 +1075,10 @@ writer::impl::impl(std::unique_ptr sink, compression_(to_parquet_compression(options.get_compression())), stats_granularity_(options.get_stats_level()), int96_timestamps(options.is_enabled_int96_timestamps()), - out_sink_(std::move(sink)), + out_sink_(), single_write_mode(mode == SingleWriteMode::YES) { + // out_sink_.push_back(std::move(sink)); if (options.get_metadata()) { table_meta = std::make_unique(*options.get_metadata()); } @@ -1048,8 +1096,9 @@ writer::impl::impl(std::unique_ptr sink, stats_granularity_(options.get_stats_level()), int96_timestamps(options.is_enabled_int96_timestamps()), single_write_mode(mode == SingleWriteMode::YES), - out_sink_(std::move(sink)) + out_sink_() { + // out_sink_.push_back(std::move(sink)); if (options.get_metadata()) { table_meta = std::make_unique(*options.get_metadata()); } @@ -1060,16 +1109,23 @@ writer::impl::~impl() { close(); } void writer::impl::init_state() { + out_sink_.push_back(data_sink::create("first.parquet")); + out_sink_.push_back(data_sink::create("second.parquet")); + current_chunk_offset.resize(out_sink_.size()); // Write file header file_header_s fhdr; fhdr.magic = parquet_magic; - out_sink_->host_write(&fhdr, sizeof(fhdr)); - current_chunk_offset = sizeof(file_header_s); + for (size_t p = 0; p < out_sink_.size(); p++) { + out_sink_[p]->host_write(&fhdr, sizeof(fhdr)); + current_chunk_offset[p] = sizeof(file_header_s); + } } -void writer::impl::write(table_view const& table) +void writer::impl::write(table_view const& table, + std::vector> partitions) { CUDF_EXPECTS(not closed, "Data has already been flushed to out and closed"); + // Verify that partitions don't overlap size_type num_rows = table.num_rows(); @@ -1107,25 +1163,34 @@ void writer::impl::write(table_view const& table) std::vector this_table_schema(schema_tree.begin(), schema_tree.end()); - if (md.version == 0) { - md.version = 1; - md.num_rows = num_rows; - md.column_order_listsize = + if (!md) { + md = std::make_unique(partitions.size()); + md->version = 1; + for (size_t i = 0; i < partitions.size(); ++i) { + md->files[i].num_rows = partitions[i].second; + } + md->column_order_listsize = (stats_granularity_ != statistics_freq::STATISTICS_NONE) ? num_columns : 0; - std::transform(table_meta->user_data.begin(), - table_meta->user_data.end(), - std::back_inserter(md.key_value_metadata), - [](auto const& kv) { - return KeyValue{kv.first, kv.second}; - }); - md.schema = this_table_schema; + + // Think about how this will be passed. Currently it is passed in table_input_metadata which is + // only passed once as part of args to writer ctor. Now this would need to be passed per sink. + // But we only need them once. Just like the list of sinks. Maybe we can + // std::transform(table_meta->user_data.begin(), + // table_meta->user_data.end(), + // std::back_inserter(md.key_value_metadata), + // [](auto const& kv) { + // return KeyValue{kv.first, kv.second}; + // }); + md->schema = this_table_schema; } else { // verify the user isn't passing mismatched tables - CUDF_EXPECTS(md.schema == this_table_schema, + CUDF_EXPECTS(md->schema == this_table_schema, "Mismatch in schema between multiple calls to write_chunk"); // increment num rows - md.num_rows += num_rows; + for (size_t i = 0; i < partitions.size(); ++i) { + md->files[i].num_rows += partitions[i].second; + } } // Create table_device_view so that corresponding column_device_view data // can be written into col_desc members @@ -1147,8 +1212,23 @@ void writer::impl::write(table_view const& table) // compression/decompression performance). using cudf::io::parquet::gpu::max_page_fragment_size; - uint32_t num_fragments = - (uint32_t)((num_rows + max_page_fragment_size - 1) / max_page_fragment_size); + std::vector num_frag_in_part; + std::transform(partitions.begin(), + partitions.end(), + std::back_inserter(num_frag_in_part), + [](auto const& part) { + return util::div_rounding_up_unsafe(part.second, max_page_fragment_size); + }); + + uint32_t num_fragments = std::reduce(num_frag_in_part.begin(), num_frag_in_part.end()); + + // TODO: better comments + std::vector part_frag_offset; // Store the idx of the first fragment in each partition + std::exclusive_scan( + num_frag_in_part.begin(), num_frag_in_part.end(), std::back_inserter(part_frag_offset), 0); + part_frag_offset.push_back(part_frag_offset.back() + num_frag_in_part.back()); + + auto d_part_frag_offset = cudf::detail::make_device_uvector_async(part_frag_offset, stream); cudf::detail::hostdevice_2dvector fragments( num_columns, num_fragments, stream); @@ -1158,37 +1238,56 @@ void writer::impl::write(table_view const& table) leaf_column_views = create_leaf_column_device_views( col_desc, *parent_column_table_device_view, stream); - init_page_fragments(fragments, col_desc, num_rows, max_page_fragment_size); + init_page_fragments( + fragments, col_desc, partitions, d_part_frag_offset, num_rows, max_page_fragment_size); } - size_t global_rowgroup_base = md.row_groups.size(); + std::vector global_rowgroup_base; + std::transform(md->files.begin(), + md->files.end(), + std::back_inserter(global_rowgroup_base), + [](auto const& part) { return part.row_groups.size(); }); // Decide row group boundaries based on uncompressed data size - size_t rowgroup_size = 0; + // size_t rowgroup_size = 0; uint32_t num_rowgroups = 0; - for (uint32_t f = 0, global_r = global_rowgroup_base, rowgroup_start = 0; f < num_fragments; - f++) { - size_t fragment_data_size = 0; - // Replace with STL algorithm to transform and sum - for (auto i = 0; i < num_columns; i++) { - fragment_data_size += fragments[i][f].fragment_data_size; - } - if (f > rowgroup_start && - (rowgroup_size + fragment_data_size > max_rowgroup_size_ || - (f + 1 - rowgroup_start) * max_page_fragment_size > max_rowgroup_rows_)) { - // update schema - md.row_groups.resize(md.row_groups.size() + 1); - md.row_groups[global_r++].num_rows = (f - rowgroup_start) * max_page_fragment_size; - num_rowgroups++; - rowgroup_start = f; - rowgroup_size = 0; - } - rowgroup_size += fragment_data_size; - if (f + 1 == num_fragments) { - // update schema - md.row_groups.resize(md.row_groups.size() + 1); - md.row_groups[global_r++].num_rows = num_rows - rowgroup_start * max_page_fragment_size; - num_rowgroups++; + + std::vector num_frag_in_rg; // TODO: Why do we need this? + std::vector num_rg_in_part(partitions.size()); + for (size_t p = 0; p < partitions.size(); ++p) { + int curr_rg_num_rows = 0; + int curr_rg_data_size = 0; + int first_frag_in_rg = part_frag_offset[p]; + int last_frag_in_part = part_frag_offset[p + 1] - 1; + for (int f = first_frag_in_rg; f <= last_frag_in_part; ++f) { + int fragment_data_size = 0; + for (auto c = 0; c < num_columns; c++) { + fragment_data_size += fragments[c][f].fragment_data_size; + } + int fragment_num_rows = fragments[0][f].num_rows; + + // If the fragment size gets larger than rg limit then break off a rg + if (f > first_frag_in_rg && // There has to be at least one fragment in row group + (curr_rg_data_size + fragment_data_size > (int)max_rowgroup_size_ || + curr_rg_num_rows + fragment_num_rows > (int)max_rowgroup_rows_)) { + auto& rg = md->files[p].row_groups.emplace_back(); + rg.num_rows = curr_rg_num_rows; + num_rowgroups++; + num_rg_in_part[p]++; + curr_rg_num_rows = 0; + curr_rg_data_size = 0; + first_frag_in_rg = f; + } + curr_rg_num_rows += fragment_num_rows; + curr_rg_data_size += fragment_data_size; + + // TODO: (wishful) refactor to consolidate with above if block + if (f == last_frag_in_part) { + auto& rg = md->files[p].row_groups.emplace_back(); + rg.num_rows = curr_rg_num_rows; + num_rowgroups++; + num_rg_in_part[p]++; + } } } @@ -1196,62 +1295,83 @@ void writer::impl::write(table_view const& table) rmm::device_uvector frag_stats(0, stream); if (stats_granularity_ != statistics_freq::STATISTICS_NONE) { frag_stats.resize(num_fragments * num_columns, stream); - if (frag_stats.size() != 0) { + if (not frag_stats.is_empty()) { auto frag_stats_2dview = device_2dspan(frag_stats.data(), num_columns, num_fragments); gather_fragment_statistics(frag_stats_2dview, fragments, col_desc, num_fragments); } } + + // TODO: My gut says there should be a way to consolidate this with rg_to_part + std::vector first_rg_in_part; + std::exclusive_scan( + num_rg_in_part.begin(), num_rg_in_part.end(), std::back_inserter(first_rg_in_part), 0); + // Initialize row groups and column chunks uint32_t num_chunks = num_rowgroups * num_columns; hostdevice_2dvector chunks(num_rowgroups, num_columns, stream); - for (uint32_t r = 0, global_r = global_rowgroup_base, f = 0, start_row = 0; r < num_rowgroups; - r++, global_r++) { - uint32_t fragments_in_chunk = (uint32_t)( - (md.row_groups[global_r].num_rows + max_page_fragment_size - 1) / max_page_fragment_size); - md.row_groups[global_r].total_byte_size = 0; - md.row_groups[global_r].columns.resize(num_columns); - for (int i = 0; i < num_columns; i++) { - gpu::EncColumnChunk* ck = &chunks[r][i]; - - *ck = {}; - ck->col_desc = col_desc.device_ptr() + i; - ck->col_desc_id = i; - ck->fragments = &fragments.device_view()[i][f]; - ck->stats = (frag_stats.size() != 0) ? frag_stats.data() + i * num_fragments + f : nullptr; - ck->start_row = start_row; - ck->num_rows = (uint32_t)md.row_groups[global_r].num_rows; - ck->first_fragment = i * num_fragments + f; - auto chunk_fragments = fragments[i].subspan(f, fragments_in_chunk); - ck->num_values = - std::accumulate(chunk_fragments.begin(), chunk_fragments.end(), 0, [](uint32_t l, auto r) { - return l + r.num_values; - }); - ck->plain_data_size = std::accumulate( - chunk_fragments.begin(), chunk_fragments.end(), 0, [](int sum, gpu::PageFragment frag) { - return sum + frag.fragment_data_size; - }); - md.row_groups[global_r].columns[i].meta_data.type = parquet_columns[i].physical_type(); - md.row_groups[global_r].columns[i].meta_data.encodings = {Encoding::PLAIN, Encoding::RLE}; - md.row_groups[global_r].columns[i].meta_data.path_in_schema = - parquet_columns[i].get_path_in_schema(); - md.row_groups[global_r].columns[i].meta_data.codec = UNCOMPRESSED; - md.row_groups[global_r].columns[i].meta_data.num_values = ck->num_values; - } - f += fragments_in_chunk; - start_row += (uint32_t)md.row_groups[global_r].num_rows; - } - auto dict_info_owner = build_chunk_dictionaries(chunks, col_desc, num_rows, stream); - for (uint32_t rg = 0, global_rg = global_rowgroup_base; rg < num_rowgroups; rg++, global_rg++) { - for (int col = 0; col < num_columns; col++) { - if (chunks.host_view()[rg][col].use_dictionary) { - md.row_groups[global_rg].columns[col].meta_data.encodings.push_back( - Encoding::PLAIN_DICTIONARY); + // TODO: alternative method is to make this a loop ovr only rg and get p using rg_to_part + for (size_t p = 0; p < partitions.size(); ++p) { + size_t f = part_frag_offset[p]; + size_t start_row = partitions[p].first; + for (int r = 0; r < num_rg_in_part[p]; r++) { + size_t global_r = global_rowgroup_base[p] + r; // Number of rowgroups already in file/part + uint32_t fragments_in_chunk = util::div_rounding_up_unsafe( + md->files[p].row_groups[global_r].num_rows, max_page_fragment_size); + md->files[p].row_groups[global_r].total_byte_size = 0; + md->files[p].row_groups[global_r].columns.resize(num_columns); + for (int c = 0; c < num_columns; c++) { + gpu::EncColumnChunk* ck = &chunks[r + first_rg_in_part[p]][c]; + + *ck = {}; + ck->col_desc = col_desc.device_ptr() + c; + ck->col_desc_id = c; + ck->fragments = &fragments.device_view()[c][f]; + ck->stats = + (not frag_stats.is_empty()) ? frag_stats.data() + c * num_fragments + f : nullptr; + ck->start_row = start_row; + ck->num_rows = (uint32_t)md->files[p].row_groups[global_r].num_rows; + ck->first_fragment = c * num_fragments + f; + auto chunk_fragments = fragments[c].subspan(f, fragments_in_chunk); + // In fragment struct, add a pointer to the chunk it belongs to + // In each fragment in chunk_fragments, update the chunk pointer here. + ck->num_values = std::accumulate( + chunk_fragments.begin(), chunk_fragments.end(), 0, [](uint32_t l, auto r) { + return l + r.num_values; + }); + ck->plain_data_size = std::accumulate( + chunk_fragments.begin(), chunk_fragments.end(), 0, [](int sum, gpu::PageFragment frag) { + return sum + frag.fragment_data_size; + }); + md->files[p].row_groups[global_r].columns[c].meta_data.type = + parquet_columns[c].physical_type(); + md->files[p].row_groups[global_r].columns[c].meta_data.encodings = {Encoding::PLAIN, + Encoding::RLE}; + md->files[p].row_groups[global_r].columns[c].meta_data.path_in_schema = + parquet_columns[c].get_path_in_schema(); + md->files[p].row_groups[global_r].columns[c].meta_data.codec = UNCOMPRESSED; + md->files[p].row_groups[global_r].columns[c].meta_data.num_values = ck->num_values; } + f += fragments_in_chunk; + start_row += (uint32_t)md->files[p].row_groups[global_r].num_rows; } } + // Pass fragments hd_vec to build_chunk_dictionaries + // auto dict_info_owner = build_chunk_dictionaries(chunks, col_desc, fragments, num_rows, stream); + // for (size_t p = 0; partitions.size(); p++) { + // for (int rg = 0; rg < num_rg_in_part[p]; rg++) { + // size_t global_rg = global_rowgroup_base[p] + rg; + // for (int col = 0; col < num_columns; col++) { + // if (chunks.host_view()[rg][col].use_dictionary) { + // md->files[p].row_groups[global_rg].columns[col].meta_data.encodings.push_back( + // Encoding::PLAIN_DICTIONARY); + // } + // } + // } + // } + // Build chunk dictionaries and count pages if (num_chunks != 0) { init_page_sizes(chunks, col_desc, num_columns); } @@ -1272,6 +1392,24 @@ void writer::impl::write(table_view const& table) "Error in getting compressed size from nvcomp"); } + // Find which partition a rg belongs to + std::vector part_end_rg; + std::inclusive_scan( + num_rg_in_part.begin(), num_rg_in_part.end(), std::back_inserter(part_end_rg)); + std::vector rg_to_part; + auto it = thrust::make_counting_iterator(0); + // std::vector range(num_rowgroups); + // std::iota(range.begin(), range.end(), 0); + // thrust::upper_bound(thrust::host, + // part_end_rg.begin(), + // part_end_rg.end(), + // range.begin(), + // range.end(), + // rg_to_part.begin()); + std::transform(it, it + num_rowgroups, std::back_inserter(rg_to_part), [&](auto i) { + return std::upper_bound(part_end_rg.begin(), part_end_rg.end(), i) - part_end_rg.begin(); + }); + // Initialize batches of rowgroups to encode (mainly to limit peak memory usage) std::vector batch_list; uint32_t num_pages = 0; @@ -1359,8 +1497,7 @@ void writer::impl::write(table_view const& table) pinned_buffer host_bfr{nullptr, cudaFreeHost}; // Encode row groups in batches - for (uint32_t b = 0, r = 0, global_r = global_rowgroup_base; b < (uint32_t)batch_list.size(); - b++) { + for (uint32_t b = 0, r = 0; b < (uint32_t)batch_list.size(); b++) { // Count pages in this batch uint32_t rnext = r + batch_list[b]; uint32_t first_page_in_batch = chunks[r][0].first_page; @@ -1380,30 +1517,33 @@ void writer::impl::write(table_view const& table) (stats_granularity_ != statistics_freq::STATISTICS_NONE) ? page_stats.data() + num_pages : nullptr); std::vector> write_tasks; - for (; r < rnext; r++, global_r++) { + for (; r < rnext; r++) { + int p = rg_to_part[r]; + int global_r = global_rowgroup_base[p] + r - first_rg_in_part[p]; for (auto i = 0; i < num_columns; i++) { gpu::EncColumnChunk* ck = &chunks[r][i]; uint8_t* dev_bfr; if (ck->is_compressed) { - md.row_groups[global_r].columns[i].meta_data.codec = compression_; - dev_bfr = ck->compressed_bfr; + md->files[p].row_groups[global_r].columns[i].meta_data.codec = compression_; + dev_bfr = ck->compressed_bfr; } else { dev_bfr = ck->uncompressed_bfr; } - if (out_sink_->is_device_write_preferred(ck->compressed_size)) { + if (out_sink_[p]->is_device_write_preferred(ck->compressed_size)) { // let the writer do what it wants to retrieve the data from the gpu. - write_tasks.push_back( - out_sink_->device_write_async(dev_bfr + ck->ck_stat_size, ck->compressed_size, stream)); + write_tasks.push_back(out_sink_[p]->device_write_async( + dev_bfr + ck->ck_stat_size, ck->compressed_size, stream)); // we still need to do a (much smaller) memcpy for the statistics. if (ck->ck_stat_size != 0) { - md.row_groups[global_r].columns[i].meta_data.statistics_blob.resize(ck->ck_stat_size); - CUDA_TRY( - cudaMemcpyAsync(md.row_groups[global_r].columns[i].meta_data.statistics_blob.data(), - dev_bfr, - ck->ck_stat_size, - cudaMemcpyDeviceToHost, - stream.value())); + md->files[p].row_groups[global_r].columns[i].meta_data.statistics_blob.resize( + ck->ck_stat_size); + CUDA_TRY(cudaMemcpyAsync( + md->files[p].row_groups[global_r].columns[i].meta_data.statistics_blob.data(), + dev_bfr, + ck->ck_stat_size, + cudaMemcpyDeviceToHost, + stream.value())); stream.synchronize(); } } else { @@ -1422,22 +1562,25 @@ void writer::impl::write(table_view const& table) cudaMemcpyDeviceToHost, stream.value())); stream.synchronize(); - out_sink_->host_write(host_bfr.get() + ck->ck_stat_size, ck->compressed_size); + out_sink_[p]->host_write(host_bfr.get() + ck->ck_stat_size, ck->compressed_size); if (ck->ck_stat_size != 0) { - md.row_groups[global_r].columns[i].meta_data.statistics_blob.resize(ck->ck_stat_size); - memcpy(md.row_groups[global_r].columns[i].meta_data.statistics_blob.data(), + md->files[p].row_groups[global_r].columns[i].meta_data.statistics_blob.resize( + ck->ck_stat_size); + memcpy(md->files[p].row_groups[global_r].columns[i].meta_data.statistics_blob.data(), host_bfr.get(), ck->ck_stat_size); } } - md.row_groups[global_r].total_byte_size += ck->compressed_size; - md.row_groups[global_r].columns[i].meta_data.data_page_offset = - current_chunk_offset + ((ck->use_dictionary) ? ck->dictionary_size : 0); - md.row_groups[global_r].columns[i].meta_data.dictionary_page_offset = - (ck->use_dictionary) ? current_chunk_offset : 0; - md.row_groups[global_r].columns[i].meta_data.total_uncompressed_size = ck->bfr_size; - md.row_groups[global_r].columns[i].meta_data.total_compressed_size = ck->compressed_size; - current_chunk_offset += ck->compressed_size; + md->files[p].row_groups[global_r].total_byte_size += ck->compressed_size; + md->files[p].row_groups[global_r].columns[i].meta_data.data_page_offset = + current_chunk_offset[p] + ((ck->use_dictionary) ? ck->dictionary_size : 0); + md->files[p].row_groups[global_r].columns[i].meta_data.dictionary_page_offset = + (ck->use_dictionary) ? current_chunk_offset[p] : 0; + md->files[p].row_groups[global_r].columns[i].meta_data.total_uncompressed_size = + ck->bfr_size; + md->files[p].row_groups[global_r].columns[i].meta_data.total_compressed_size = + ck->compressed_size; + current_chunk_offset[p] += ck->compressed_size; } } for (auto const& task : write_tasks) { @@ -1451,35 +1594,39 @@ std::unique_ptr> writer::impl::close( { if (closed) { return nullptr; } closed = true; - CompactProtocolWriter cpw(&buffer_); - file_ender_s fendr; - buffer_.resize(0); - fendr.footer_len = static_cast(cpw.write(md)); - fendr.magic = parquet_magic; - out_sink_->host_write(buffer_.data(), buffer_.size()); - out_sink_->host_write(&fendr, sizeof(fendr)); - out_sink_->flush(); - - // Optionally output raw file metadata with the specified column chunk file path - if (column_chunks_file_path.length() > 0) { - file_header_s fhdr = {parquet_magic}; - buffer_.resize(0); - buffer_.insert(buffer_.end(), - reinterpret_cast(&fhdr), - reinterpret_cast(&fhdr) + sizeof(fhdr)); - for (auto& rowgroup : md.row_groups) { - for (auto& col : rowgroup.columns) { - col.file_path = column_chunks_file_path; - } - } - fendr.footer_len = static_cast(cpw.write(md)); - buffer_.insert(buffer_.end(), - reinterpret_cast(&fendr), - reinterpret_cast(&fendr) + sizeof(fendr)); - return std::make_unique>(std::move(buffer_)); - } else { - return {nullptr}; + for (size_t p = 0; p < out_sink_.size(); p++) { + std::vector buffer; + CompactProtocolWriter cpw(&buffer); + file_ender_s fendr; + buffer.resize(0); + fendr.footer_len = static_cast(cpw.write(md->get_metadata(p))); + fendr.magic = parquet_magic; + out_sink_[p]->host_write(buffer.data(), buffer.size()); + out_sink_[p]->host_write(&fendr, sizeof(fendr)); + out_sink_[p]->flush(); } + + // // Optionally output raw file metadata with the specified column chunk file path + // if (column_chunks_file_path.length() > 0) { + // file_header_s fhdr = {parquet_magic}; + // buffer_.resize(0); + // buffer_.insert(buffer_.end(), + // reinterpret_cast(&fhdr), + // reinterpret_cast(&fhdr) + sizeof(fhdr)); + // for (auto& rowgroup : md.row_groups) { + // for (auto& col : rowgroup.columns) { + // col.file_path = column_chunks_file_path; + // } + // } + // fendr.footer_len = static_cast(cpw.write(md)); + // buffer_.insert(buffer_.end(), + // reinterpret_cast(&fendr), + // reinterpret_cast(&fendr) + sizeof(fendr)); + // return std::make_unique>(std::move(buffer_)); + // } else { + // return {nullptr}; + // } + return nullptr; } // Forward to implementation @@ -1505,7 +1652,11 @@ writer::writer(std::unique_ptr sink, writer::~writer() = default; // Forward to implementation -void writer::write(table_view const& table) { _impl->write(table); } +void writer::write(table_view const& table) +{ + _impl->write(table, {{1, 2}, {4, 1}}); + // _impl->write(table, {{10, 20 * 1024}, {20 * 1024 + 10, 30 * 1024}}); +} // Forward to implementation std::unique_ptr> writer::close(std::string const& column_chunks_file_path) diff --git a/cpp/src/io/parquet/writer_impl.hpp b/cpp/src/io/parquet/writer_impl.hpp index c7cdf8effd1..a6d996b7a74 100644 --- a/cpp/src/io/parquet/writer_impl.hpp +++ b/cpp/src/io/parquet/writer_impl.hpp @@ -45,6 +45,7 @@ namespace detail { namespace parquet { // Forward internal classes struct parquet_column_view; +struct per_file_metadata; using namespace cudf::io::parquet; using namespace cudf::io; @@ -110,7 +111,7 @@ class writer::impl { * * @param[in] table The table information to be written */ - void write(table_view const& table); + void write(table_view const& table, std::vector> partitions); /** * @brief Finishes the chunked/streamed write process. @@ -132,6 +133,8 @@ class writer::impl { */ void init_page_fragments(hostdevice_2dvector& frag, device_span col_desc, + std::vector> partitions, + device_span part_frag_offset, uint32_t num_rows, uint32_t fragment_size); @@ -216,19 +219,19 @@ class writer::impl { statistics_freq stats_granularity_ = statistics_freq::STATISTICS_NONE; bool int96_timestamps = false; // Overall file metadata. Filled in during the process and written during write_chunked_end() - cudf::io::parquet::FileMetaData md; + std::unique_ptr md; // optional user metadata std::unique_ptr table_meta; // to track if the output has been written to sink bool closed = false; // current write position for rowgroups/chunks - std::size_t current_chunk_offset; + std::vector current_chunk_offset; // special parameter only used by detail::write() to indicate that we are guaranteeing // a single table write. this enables some internal optimizations. bool const single_write_mode = true; std::vector buffer_; - std::unique_ptr out_sink_; + std::vector> out_sink_; }; } // namespace parquet diff --git a/cpp/tests/io/parquet_test.cpp b/cpp/tests/io/parquet_test.cpp index 4ac3e7a2ec3..a4225e2ed3c 100644 --- a/cpp/tests/io/parquet_test.cpp +++ b/cpp/tests/io/parquet_test.cpp @@ -573,18 +573,19 @@ TEST_F(ParquetWriterTest, Strings) expected_metadata.column_metadata[1].set_name("col_string"); expected_metadata.column_metadata[2].set_name("col_another"); - auto filepath = temp_env->get_temp_filepath("Strings.parquet"); + auto filepath = ("Strings.parquet"); cudf_io::parquet_writer_options out_opts = cudf_io::parquet_writer_options::builder(cudf_io::sink_info{filepath}, expected->view()) - .metadata(&expected_metadata); + .metadata(&expected_metadata) + .stats_level(cudf::io::STATISTICS_NONE); cudf_io::write_parquet(out_opts); - cudf_io::parquet_reader_options in_opts = - cudf_io::parquet_reader_options::builder(cudf_io::source_info{filepath}); - auto result = cudf_io::read_parquet(in_opts); + // cudf_io::parquet_reader_options in_opts = + // cudf_io::parquet_reader_options::builder(cudf_io::source_info{filepath}); + // auto result = cudf_io::read_parquet(in_opts); - CUDF_TEST_EXPECT_TABLES_EQUAL(expected->view(), result.tbl->view()); - cudf::test::expect_metadata_equal(expected_metadata, result.metadata); + // CUDF_TEST_EXPECT_TABLES_EQUAL(expected->view(), result.tbl->view()); + // cudf::test::expect_metadata_equal(expected_metadata, result.metadata); } TEST_F(ParquetWriterTest, SlicedTable) @@ -2113,7 +2114,7 @@ TEST_F(ParquetWriterStressTest, LargeTableGoodCompression) // exercises multiple rowgroups srand(31337); - auto expected = create_compressible_fixed_table(16, 4 * 1024 * 1024, 128 * 1024, false); + auto expected = create_compressible_fixed_table(3, 4 * 1024 * 1024, 128 * 1024, false); // write out using the custom sink (which uses device writes) cudf_io::parquet_writer_options args = From 21dc54b220aaf8c42038e82bad401c4effaa09df Mon Sep 17 00:00:00 2001 From: Devavret Makkar Date: Tue, 23 Nov 2021 17:47:20 +0530 Subject: [PATCH 02/71] multiple sink API --- cpp/include/cudf/io/data_sink.hpp | 16 +++++++++++ cpp/include/cudf/io/detail/parquet.hpp | 4 +-- cpp/include/cudf/io/types.hpp | 32 +++++++++++++++------ cpp/src/io/functions.cpp | 39 ++++++++++++++++---------- cpp/src/io/parquet/writer_impl.cu | 20 ++++++------- cpp/src/io/parquet/writer_impl.hpp | 8 +++--- cpp/tests/io/parquet_test.cpp | 5 ++-- 7 files changed, 82 insertions(+), 42 deletions(-) diff --git a/cpp/include/cudf/io/data_sink.hpp b/cpp/include/cudf/io/data_sink.hpp index 42421aed716..2c1966ee6ba 100644 --- a/cpp/include/cudf/io/data_sink.hpp +++ b/cpp/include/cudf/io/data_sink.hpp @@ -69,6 +69,22 @@ class data_sink { */ static std::unique_ptr create(cudf::io::data_sink* const user_sink); + /** + * @brief Creates a vector of data sinks, one per element in the input vector. + * + * @param[in] args vector of parameters + */ + template + static std::vector> create(std::vector const& args) + { + std::vector> sinks; + sinks.reserve(args.size()); + std::transform(args.cbegin(), args.cend(), std::back_inserter(sinks), [](auto const& arg) { + return data_sink::create(arg); + }); + return sinks; + } + /** * @brief Base class destructor */ diff --git a/cpp/include/cudf/io/detail/parquet.hpp b/cpp/include/cudf/io/detail/parquet.hpp index 98922ad10a4..c6f48977786 100644 --- a/cpp/include/cudf/io/detail/parquet.hpp +++ b/cpp/include/cudf/io/detail/parquet.hpp @@ -97,7 +97,7 @@ class writer { * @param stream CUDA stream used for device memory operations and kernel launches * @param mr Device memory resource to use for device memory allocation */ - explicit writer(std::unique_ptr sink, + explicit writer(std::vector> sinks, parquet_writer_options const& options, SingleWriteMode mode, rmm::cuda_stream_view stream, @@ -114,7 +114,7 @@ class writer { * * @return A parquet-compatible blob that contains the data for all rowgroups in the list */ - explicit writer(std::unique_ptr sink, + explicit writer(std::vector> sinks, chunked_parquet_writer_options const& options, SingleWriteMode mode, rmm::cuda_stream_view stream, diff --git a/cpp/include/cudf/io/types.hpp b/cpp/include/cudf/io/types.hpp index ac965e2d416..401b9635847 100644 --- a/cpp/include/cudf/io/types.hpp +++ b/cpp/include/cudf/io/types.hpp @@ -191,19 +191,35 @@ struct source_info { * @brief Destination information for write interfaces */ struct sink_info { - io_type type = io_type::VOID; - std::string filepath; - std::vector* buffer = nullptr; - cudf::io::data_sink* user_sink = nullptr; + io_type type = io_type::VOID; + size_t num_void_sinks = 1; + std::vector filepaths; + std::vector*> buffers; // TODO: perhaps we can repurpose host_buffer. ask VM + std::vector user_sinks; sink_info() = default; + sink_info(size_t num_void_sinks) : type(io_type::VOID), num_void_sinks(num_void_sinks) {} - explicit sink_info(const std::string& file_path) : type(io_type::FILEPATH), filepath(file_path) {} + explicit sink_info(std::vector const& file_paths) + : type(io_type::FILEPATH), filepaths(file_paths) + { + } + explicit sink_info(std::string const& file_path) : type(io_type::FILEPATH), filepaths({file_path}) + { + } - explicit sink_info(std::vector* buffer) : type(io_type::HOST_BUFFER), buffer(buffer) {} + explicit sink_info(std::vector*> const& buffers) + : type(io_type::HOST_BUFFER), buffers(buffers) + { + } + explicit sink_info(std::vector* buffer) : type(io_type::HOST_BUFFER), buffers({buffer}) {} - explicit sink_info(class cudf::io::data_sink* user_sink_) - : type(io_type::USER_IMPLEMENTED), user_sink(user_sink_) + explicit sink_info(std::vector const& user_sinks) + : type(io_type::USER_IMPLEMENTED), user_sinks(user_sinks) + { + } + explicit sink_info(class cudf::io::data_sink* user_sink) + : type(io_type::USER_IMPLEMENTED), user_sinks({user_sink}) { } }; diff --git a/cpp/src/io/functions.cpp b/cpp/src/io/functions.cpp index 402e212f07b..f69f634fa6f 100644 --- a/cpp/src/io/functions.cpp +++ b/cpp/src/io/functions.cpp @@ -125,13 +125,19 @@ std::vector> make_datasources(source_info } } -std::unique_ptr make_datasink(sink_info const& info) +std::vector> make_datasinks(sink_info const& info) { switch (info.type) { - case io_type::FILEPATH: return cudf::io::data_sink::create(info.filepath); - case io_type::HOST_BUFFER: return cudf::io::data_sink::create(info.buffer); - case io_type::VOID: return cudf::io::data_sink::create(); - case io_type::USER_IMPLEMENTED: return cudf::io::data_sink::create(info.user_sink); + case io_type::FILEPATH: return cudf::io::data_sink::create(info.filepaths); + case io_type::HOST_BUFFER: return cudf::io::data_sink::create(info.buffers); + case io_type::VOID: { + std::vector> sinks; + for (size_t i = 0; i < info.num_void_sinks; ++i) { + sinks.push_back(cudf::io::data_sink::create()); + } + return sinks; + } + case io_type::USER_IMPLEMENTED: return cudf::io::data_sink::create(info.user_sinks); default: CUDF_FAIL("Unsupported sink type"); } } @@ -218,10 +224,11 @@ void write_csv(csv_writer_options const& options, rmm::mr::device_memory_resourc { using namespace cudf::io::detail; - auto sink = make_datasink(options.get_sink()); + // TODO: protect against multiple sinks until we support it in CSV + auto sinks = make_datasinks(options.get_sink()); return csv::write_csv( // - sink.get(), + sinks[0].get(), options.get_table(), options.get_metadata(), options, @@ -350,9 +357,10 @@ void write_orc(orc_writer_options const& options, rmm::mr::device_memory_resourc CUDF_FUNC_RANGE(); - auto sink = make_datasink(options.get_sink()); + // TODO: protect against multiple sinks until we support it in ORC + auto sinks = make_datasinks(options.get_sink()); auto writer = std::make_unique( - std::move(sink), options, io_detail::SingleWriteMode::YES, rmm::cuda_stream_default, mr); + std::move(sinks[0]), options, io_detail::SingleWriteMode::YES, rmm::cuda_stream_default, mr); writer->write(options.get_table()); } @@ -365,10 +373,11 @@ orc_chunked_writer::orc_chunked_writer(chunked_orc_writer_options const& options { namespace io_detail = cudf::io::detail; - auto sink = make_datasink(options.get_sink()); + // TODO: protect against multiple sinks until we support it in ORC + auto sinks = make_datasinks(options.get_sink()); writer = std::make_unique( - std::move(sink), options, io_detail::SingleWriteMode::NO, rmm::cuda_stream_default, mr); + std::move(sinks[0]), options, io_detail::SingleWriteMode::NO, rmm::cuda_stream_default, mr); } /** @@ -444,9 +453,9 @@ std::unique_ptr> write_parquet(parquet_writer_options const CUDF_FUNC_RANGE(); - auto sink = make_datasink(options.get_sink()); + auto sinks = make_datasinks(options.get_sink()); auto writer = std::make_unique( - std::move(sink), options, io_detail::SingleWriteMode::YES, rmm::cuda_stream_default, mr); + std::move(sinks), options, io_detail::SingleWriteMode::YES, rmm::cuda_stream_default, mr); writer->write(options.get_table()); @@ -461,10 +470,10 @@ parquet_chunked_writer::parquet_chunked_writer(chunked_parquet_writer_options co { namespace io_detail = cudf::io::detail; - auto sink = make_datasink(options.get_sink()); + auto sinks = make_datasinks(options.get_sink()); writer = std::make_unique( - std::move(sink), options, io_detail::SingleWriteMode::NO, rmm::cuda_stream_default, mr); + std::move(sinks), options, io_detail::SingleWriteMode::NO, rmm::cuda_stream_default, mr); } /** diff --git a/cpp/src/io/parquet/writer_impl.cu b/cpp/src/io/parquet/writer_impl.cu index bb6317d19a0..bd80a1a6d12 100644 --- a/cpp/src/io/parquet/writer_impl.cu +++ b/cpp/src/io/parquet/writer_impl.cu @@ -1066,7 +1066,7 @@ void writer::impl::encode_pages(hostdevice_2dvector& chunks stream.synchronize(); } -writer::impl::impl(std::unique_ptr sink, +writer::impl::impl(std::vector> sinks, parquet_writer_options const& options, SingleWriteMode mode, rmm::cuda_stream_view stream, @@ -1078,8 +1078,8 @@ writer::impl::impl(std::unique_ptr sink, compression_(to_parquet_compression(options.get_compression())), stats_granularity_(options.get_stats_level()), int96_timestamps(options.is_enabled_int96_timestamps()), - out_sink_(), - single_write_mode(mode == SingleWriteMode::YES) + single_write_mode(mode == SingleWriteMode::YES), + out_sink_(std::move(sinks)) { // out_sink_.push_back(std::move(sink)); if (options.get_metadata()) { @@ -1088,7 +1088,7 @@ writer::impl::impl(std::unique_ptr sink, init_state(); } -writer::impl::impl(std::unique_ptr sink, +writer::impl::impl(std::vector> sinks, chunked_parquet_writer_options const& options, SingleWriteMode mode, rmm::cuda_stream_view stream, @@ -1101,7 +1101,7 @@ writer::impl::impl(std::unique_ptr sink, stats_granularity_(options.get_stats_level()), int96_timestamps(options.is_enabled_int96_timestamps()), single_write_mode(mode == SingleWriteMode::YES), - out_sink_() + out_sink_(std::move(sinks)) { // out_sink_.push_back(std::move(sink)); if (options.get_metadata()) { @@ -1114,8 +1114,6 @@ writer::impl::~impl() { close(); } void writer::impl::init_state() { - out_sink_.push_back(data_sink::create("first.parquet")); - out_sink_.push_back(data_sink::create("second.parquet")); current_chunk_offset.resize(out_sink_.size()); // Write file header file_header_s fhdr; @@ -1638,21 +1636,21 @@ std::unique_ptr> writer::impl::close( } // Forward to implementation -writer::writer(std::unique_ptr sink, +writer::writer(std::vector> sinks, parquet_writer_options const& options, SingleWriteMode mode, rmm::cuda_stream_view stream, rmm::mr::device_memory_resource* mr) - : _impl(std::make_unique(std::move(sink), options, mode, stream, mr)) + : _impl(std::make_unique(std::move(sinks), options, mode, stream, mr)) { } -writer::writer(std::unique_ptr sink, +writer::writer(std::vector> sinks, chunked_parquet_writer_options const& options, SingleWriteMode mode, rmm::cuda_stream_view stream, rmm::mr::device_memory_resource* mr) - : _impl(std::make_unique(std::move(sink), options, mode, stream, mr)) + : _impl(std::make_unique(std::move(sinks), options, mode, stream, mr)) { } diff --git a/cpp/src/io/parquet/writer_impl.hpp b/cpp/src/io/parquet/writer_impl.hpp index 73e551e354c..838cb373ddc 100644 --- a/cpp/src/io/parquet/writer_impl.hpp +++ b/cpp/src/io/parquet/writer_impl.hpp @@ -61,13 +61,13 @@ class writer::impl { /** * @brief Constructor with writer options. * - * @param sink data_sink for storing dataset + * @param sink data_sink's for storing dataset * @param options Settings for controlling behavior * @param mode Option to write at once or in chunks * @param stream CUDA stream used for device memory operations and kernel launches * @param mr Device memory resource to use for device memory allocation */ - explicit impl(std::unique_ptr sink, + explicit impl(std::vector> sinks, parquet_writer_options const& options, SingleWriteMode mode, rmm::cuda_stream_view stream, @@ -76,13 +76,13 @@ class writer::impl { /** * @brief Constructor with chunked writer options. * - * @param sink data_sink for storing dataset + * @param sink data_sink's for storing dataset * @param options Settings for controlling behavior * @param mode Option to write at once or in chunks * @param stream CUDA stream used for device memory operations and kernel launches * @param mr Device memory resource to use for device memory allocation */ - explicit impl(std::unique_ptr sink, + explicit impl(std::vector> sinks, chunked_parquet_writer_options const& options, SingleWriteMode mode, rmm::cuda_stream_view stream, diff --git a/cpp/tests/io/parquet_test.cpp b/cpp/tests/io/parquet_test.cpp index 1cfdc228ca0..c024d1eba86 100644 --- a/cpp/tests/io/parquet_test.cpp +++ b/cpp/tests/io/parquet_test.cpp @@ -573,9 +573,10 @@ TEST_F(ParquetWriterTest, Strings) expected_metadata.column_metadata[1].set_name("col_string"); expected_metadata.column_metadata[2].set_name("col_another"); - auto filepath = ("Strings.parquet"); cudf_io::parquet_writer_options out_opts = - cudf_io::parquet_writer_options::builder(cudf_io::sink_info{filepath}, expected->view()) + cudf_io::parquet_writer_options::builder( + cudf_io::sink_info(std::vector{"first.parquet", "second.parquet"}), + expected->view()) .metadata(&expected_metadata) .stats_level(cudf::io::STATISTICS_NONE); cudf_io::write_parquet(out_opts); From d947abd2a20bfb382ca324b77820af40c715efde Mon Sep 17 00:00:00 2001 From: Devavret Makkar Date: Wed, 24 Nov 2021 02:49:12 +0530 Subject: [PATCH 03/71] partitions in write parquet API --- cpp/include/cudf/io/detail/parquet.hpp | 3 ++- cpp/include/cudf/io/parquet.hpp | 35 +++++++++++++++++++++++++- cpp/src/io/functions.cpp | 7 +++--- cpp/src/io/parquet/writer_impl.cu | 5 ++-- cpp/tests/io/parquet_test.cpp | 1 + 5 files changed, 44 insertions(+), 7 deletions(-) diff --git a/cpp/include/cudf/io/detail/parquet.hpp b/cpp/include/cudf/io/detail/parquet.hpp index c6f48977786..24cc4bc3294 100644 --- a/cpp/include/cudf/io/detail/parquet.hpp +++ b/cpp/include/cudf/io/detail/parquet.hpp @@ -130,7 +130,8 @@ class writer { * * @param[in] table The table information to be written */ - void write(table_view const& table); + void write(table_view const& table, + std::vector> const& partitions = {}); /** * @brief Finishes the chunked/streamed write process. diff --git a/cpp/include/cudf/io/parquet.hpp b/cpp/include/cudf/io/parquet.hpp index 88cf7416506..7d0715883c3 100644 --- a/cpp/include/cudf/io/parquet.hpp +++ b/cpp/include/cudf/io/parquet.hpp @@ -394,6 +394,8 @@ class parquet_writer_options { statistics_freq _stats_level = statistics_freq::STATISTICS_ROWGROUP; // Sets of columns to output table_view _table; + // Partitions described in a {start_row, num_rows} pairs + std::vector> _partitions; // Optional associated metadata table_input_metadata const* _metadata = nullptr; // Parquet writer can write INT96 or TIMESTAMP_MICROS. Defaults to TIMESTAMP_MICROS. @@ -464,6 +466,11 @@ class parquet_writer_options { */ table_view get_table() const { return _table; } + /** + * @brief Returns partitions. + */ + std::vector> get_partitions() const { return _partitions; } + /** * @brief Returns associated metadata. */ @@ -489,6 +496,16 @@ class parquet_writer_options { */ auto get_row_group_size_rows() const { return _row_group_size_rows; } + /** + * @brief Sets metadata. + * + * @param partitions Associated metadata. + */ + void set_partitions(std::vector> const& partitions) + { + _partitions = partitions; + } + /** * @brief Sets metadata. * @@ -573,6 +590,19 @@ class parquet_writer_options_builder { { } + /** + * @brief Sets partitions in parquet_writer_options. + * + * @param partitions Partitions of input table in {start_row, num_rows} pairs. + * @return this for chaining. + */ + parquet_writer_options_builder& partitions( + std::vector> partitions) + { + options._partitions = partitions; + return *this; + } + /** * @brief Sets metadata in parquet_writer_options. * @@ -988,9 +1018,12 @@ class parquet_chunked_writer { * @brief Writes table to output. * * @param[in] table Table that needs to be written + * @param[in] partitions Optional partitions to divide the table into. If specified, must be same + * size as number of sinks. * @return returns reference of the class object */ - parquet_chunked_writer& write(table_view const& table); + parquet_chunked_writer& write( + table_view const& table, std::vector> const& partitions = {}); /** * @brief Finishes the chunked/streamed write process. diff --git a/cpp/src/io/functions.cpp b/cpp/src/io/functions.cpp index f69f634fa6f..a755c2c937c 100644 --- a/cpp/src/io/functions.cpp +++ b/cpp/src/io/functions.cpp @@ -457,7 +457,7 @@ std::unique_ptr> write_parquet(parquet_writer_options const auto writer = std::make_unique( std::move(sinks), options, io_detail::SingleWriteMode::YES, rmm::cuda_stream_default, mr); - writer->write(options.get_table()); + writer->write(options.get_table(), options.get_partitions()); return writer->close(options.get_column_chunks_file_path()); } @@ -479,11 +479,12 @@ parquet_chunked_writer::parquet_chunked_writer(chunked_parquet_writer_options co /** * @copydoc cudf::io::parquet_chunked_writer::write */ -parquet_chunked_writer& parquet_chunked_writer::write(table_view const& table) +parquet_chunked_writer& parquet_chunked_writer::write( + table_view const& table, std::vector> const& partitions) { CUDF_FUNC_RANGE(); - writer->write(table); + writer->write(table, partitions); return *this; } diff --git a/cpp/src/io/parquet/writer_impl.cu b/cpp/src/io/parquet/writer_impl.cu index bd80a1a6d12..3373a45d16e 100644 --- a/cpp/src/io/parquet/writer_impl.cu +++ b/cpp/src/io/parquet/writer_impl.cu @@ -1658,9 +1658,10 @@ writer::writer(std::vector> sinks, writer::~writer() = default; // Forward to implementation -void writer::write(table_view const& table) +void writer::write(table_view const& table, + std::vector> const& partitions) { - _impl->write(table, {{1, 2}, {4, 1}}); + _impl->write(table, partitions); // _impl->write(table, {{10, 20 * 1024}, {20 * 1024 + 10, 30 * 1024}}); } diff --git a/cpp/tests/io/parquet_test.cpp b/cpp/tests/io/parquet_test.cpp index c024d1eba86..b1ab9d84867 100644 --- a/cpp/tests/io/parquet_test.cpp +++ b/cpp/tests/io/parquet_test.cpp @@ -577,6 +577,7 @@ TEST_F(ParquetWriterTest, Strings) cudf_io::parquet_writer_options::builder( cudf_io::sink_info(std::vector{"first.parquet", "second.parquet"}), expected->view()) + .partitions({{1, 2}, {4, 1}}) .metadata(&expected_metadata) .stats_level(cudf::io::STATISTICS_NONE); cudf_io::write_parquet(out_opts); From 360bf87de66a0a7c6eae33e30efd83768c5de784 Mon Sep 17 00:00:00 2001 From: Devavret Makkar Date: Thu, 25 Nov 2021 00:13:04 +0530 Subject: [PATCH 04/71] Fix a bug in frag causing incorrect num rows --- cpp/src/io/parquet/parquet_gpu.hpp | 2 +- cpp/src/io/parquet/writer_impl.cu | 1 - cpp/tests/io/parquet_test.cpp | 14 ++++++++------ 3 files changed, 9 insertions(+), 8 deletions(-) diff --git a/cpp/src/io/parquet/parquet_gpu.hpp b/cpp/src/io/parquet/parquet_gpu.hpp index 6eedff355c1..8131a1d301e 100644 --- a/cpp/src/io/parquet/parquet_gpu.hpp +++ b/cpp/src/io/parquet/parquet_gpu.hpp @@ -269,7 +269,7 @@ struct PageFragment { //!< non-leaf level // Add a start_row member because fragments no longer 5000 rows each - uint16_t start_row; //!< First row in fragment + size_type start_row; //!< First row in fragment uint16_t num_rows; //!< Number of rows in fragment uint16_t num_dict_vals; //!< Number of unique dictionary entries }; diff --git a/cpp/src/io/parquet/writer_impl.cu b/cpp/src/io/parquet/writer_impl.cu index 3373a45d16e..6f4a067dd37 100644 --- a/cpp/src/io/parquet/writer_impl.cu +++ b/cpp/src/io/parquet/writer_impl.cu @@ -1662,7 +1662,6 @@ void writer::write(table_view const& table, std::vector> const& partitions) { _impl->write(table, partitions); - // _impl->write(table, {{10, 20 * 1024}, {20 * 1024 + 10, 30 * 1024}}); } // Forward to implementation diff --git a/cpp/tests/io/parquet_test.cpp b/cpp/tests/io/parquet_test.cpp index b1ab9d84867..16f3c2d8cbb 100644 --- a/cpp/tests/io/parquet_test.cpp +++ b/cpp/tests/io/parquet_test.cpp @@ -2116,17 +2116,19 @@ TEST_F(ParquetWriterStressTest, LargeTableGoodCompression) // exercises multiple rowgroups srand(31337); - auto expected = create_compressible_fixed_table(3, 4 * 1024 * 1024, 128 * 1024, false); + auto expected = create_compressible_fixed_table(16, 4 * 1024 * 1024, 10, false); // write out using the custom sink (which uses device writes) cudf_io::parquet_writer_options args = - cudf_io::parquet_writer_options::builder(cudf_io::sink_info{&custom_sink}, *expected); + cudf_io::parquet_writer_options::builder( + cudf_io::sink_info(std::vector{"first.parquet", "second.parquet"}), *expected) + .partitions({{10, 256 * 1024}, {256 * 1024 + 7, 1024 * 1024}}); cudf_io::write_parquet(args); - cudf_io::parquet_reader_options custom_args = - cudf_io::parquet_reader_options::builder(cudf_io::source_info{mm_buf.data(), mm_buf.size()}); - auto custom_tbl = cudf_io::read_parquet(custom_args); - CUDF_TEST_EXPECT_TABLES_EQUAL(custom_tbl.tbl->view(), expected->view()); + // cudf_io::parquet_reader_options custom_args = + // cudf_io::parquet_reader_options::builder(cudf_io::source_info{mm_buf.data(), mm_buf.size()}); + // auto custom_tbl = cudf_io::read_parquet(custom_args); + // CUDF_TEST_EXPECT_TABLES_EQUAL(custom_tbl.tbl->view(), expected->view()); } TEST_F(ParquetWriterStressTest, LargeTableWithValids) From d45450762c435124730a5060a7245d5e4c176040 Mon Sep 17 00:00:00 2001 From: Devavret Makkar Date: Thu, 25 Nov 2021 16:50:15 +0530 Subject: [PATCH 05/71] Dict encoding changes. Dict kernels now use frags --- cpp/src/io/parquet/chunk_dict.cu | 96 ++++++++++++++++++------------ cpp/src/io/parquet/parquet_gpu.hpp | 12 ++-- cpp/src/io/parquet/writer_impl.cu | 30 ++++++---- cpp/tests/io/parquet_test.cpp | 5 +- 4 files changed, 85 insertions(+), 58 deletions(-) diff --git a/cpp/src/io/parquet/chunk_dict.cu b/cpp/src/io/parquet/chunk_dict.cu index e6a9041d600..f73af258109 100644 --- a/cpp/src/io/parquet/chunk_dict.cu +++ b/cpp/src/io/parquet/chunk_dict.cu @@ -95,33 +95,43 @@ struct map_find_fn { template __global__ void __launch_bounds__(block_size, 1) populate_chunk_hash_maps_kernel(cudf::detail::device_2dspan chunks, + cudf::detail::device_2dspan frags, size_type num_rows) { auto col_idx = blockIdx.y; auto block_x = blockIdx.x; auto t = threadIdx.x; - - auto start_row = - block_x * - max_page_fragment_size; // This is fragment size. all chunks are multiple of these many rows. - size_type end_row = min(start_row + max_page_fragment_size, num_rows); - + auto frag = frags[col_idx][block_x]; + auto chunk = frag.chunk; + + // auto start_row = + // block_x * + // max_page_fragment_size; // This is fragment size. all chunks are multiple of these many + // rows. + // size_type end_row = min(start_row + max_page_fragment_size, num_rows); + size_type start_row = frag.start_row; + size_type end_row = frag.start_row + frag.num_rows; + + // TODO: s_chunk maybe not needed __shared__ EncColumnChunk* s_chunk; __shared__ parquet_column_device_view s_col; __shared__ size_type s_start_value_idx; __shared__ size_type s_num_values; + + // TODO: cleanup. No need to get chunk like this if (t == 0) { - // Find the chunk this block is a part of - size_type num_rowgroups = chunks.size().first; - size_type rg_idx = 0; - while (rg_idx < num_rowgroups) { - if (auto ck = chunks[rg_idx][col_idx]; - start_row >= ck.start_row and start_row < ck.start_row + ck.num_rows) { - break; - } - ++rg_idx; - } - s_chunk = &chunks[rg_idx][col_idx]; + // // Find the chunk this block is a part of + // size_type num_rowgroups = chunks.size().first; + // size_type rg_idx = 0; + // while (rg_idx < num_rowgroups) { + // if (auto ck = chunks[rg_idx][col_idx]; + // start_row >= ck.start_row and start_row < ck.start_row + ck.num_rows) { + // break; + // } + // ++rg_idx; + // } + // s_chunk = &chunks[rg_idx][col_idx]; + s_chunk = chunk; s_col = *(s_chunk->col_desc); } __syncthreads(); @@ -245,14 +255,19 @@ __global__ void __launch_bounds__(block_size, 1) template __global__ void __launch_bounds__(block_size, 1) get_dictionary_indices_kernel(cudf::detail::device_2dspan chunks, + cudf::detail::device_2dspan frags, size_type num_rows) { auto col_idx = blockIdx.y; auto block_x = blockIdx.x; auto t = threadIdx.x; + auto frag = frags[col_idx][block_x]; + // auto chunk = frag.chunk; - size_type start_row = block_x * max_page_fragment_size; - size_type end_row = min(start_row + max_page_fragment_size, num_rows); + // size_type start_row = block_x * max_page_fragment_size; + // size_type end_row = min(start_row + max_page_fragment_size, num_rows); + size_type start_row = frag.start_row; + size_type end_row = frag.start_row + frag.num_rows; __shared__ EncColumnChunk s_chunk; __shared__ parquet_column_device_view s_col; @@ -261,17 +276,18 @@ __global__ void __launch_bounds__(block_size, 1) __shared__ size_type s_num_values; if (t == 0) { - // Find the chunk this block is a part of - size_type num_rowgroups = chunks.size().first; - size_type rg_idx = 0; - while (rg_idx < num_rowgroups) { - if (auto ck = chunks[rg_idx][col_idx]; - start_row >= ck.start_row and start_row < ck.start_row + ck.num_rows) { - break; - } - ++rg_idx; - } - s_chunk = chunks[rg_idx][col_idx]; + // // Find the chunk this block is a part of + // size_type num_rowgroups = chunks.size().first; + // size_type rg_idx = 0; + // while (rg_idx < num_rowgroups) { + // if (auto ck = chunks[rg_idx][col_idx]; + // start_row >= ck.start_row and start_row < ck.start_row + ck.num_rows) { + // break; + // } + // ++rg_idx; + // } + // s_chunk = chunks[rg_idx][col_idx]; + s_chunk = *frag.chunk; s_col = *(s_chunk.col_desc); // Find the bounds of values in leaf column to be inserted into the map for current chunk @@ -340,9 +356,11 @@ void populate_chunk_hash_maps(cudf::detail::device_2dspan chunks rmm::cuda_stream_view stream) { constexpr int block_size = 256; - auto const grid_x = cudf::detail::grid_1d(num_rows, max_page_fragment_size); - auto const num_columns = chunks.size().second; - dim3 const dim_grid(grid_x.num_blocks, num_columns); + // auto const grid_x = cudf::detail::grid_1d(num_rows, max_page_fragment_size); + // auto const num_columns = chunks.size().second; + // dim3 const dim_grid(grid_x.num_blocks, num_columns); + // TODO: Is there any perf implications if the kernel is launched with x = cols, y = rows/frags? + dim3 const dim_grid(frags.size().second, frags.size().first); // Convert to a per-fragment kernel. It is like that already and I know we cannot avoid fragments // anymore. The only other alternative is using row_bit_count to find per-row size and then use it @@ -350,7 +368,7 @@ void populate_chunk_hash_maps(cudf::detail::device_2dspan chunks // future we can remove fragments and allow rowgroups to have less than 5000 rows but it's not // important right now populate_chunk_hash_maps_kernel - <<>>(chunks, num_rows); + <<>>(chunks, frags, num_rows); } void collect_map_entries(device_span chunks, rmm::cuda_stream_view stream) @@ -360,16 +378,18 @@ void collect_map_entries(device_span chunks, rmm::cuda_stream_vi } void get_dictionary_indices(cudf::detail::device_2dspan chunks, + cudf::detail::device_2dspan frags, size_type num_rows, rmm::cuda_stream_view stream) { constexpr int block_size = 256; - auto const grid_x = cudf::detail::grid_1d(num_rows, max_page_fragment_size); - auto const num_columns = chunks.size().second; - dim3 const dim_grid(grid_x.num_blocks, num_columns); + // auto const grid_x = cudf::detail::grid_1d(num_rows, max_page_fragment_size); + // auto const num_columns = chunks.size().second; + // dim3 const dim_grid(grid_x.num_blocks, num_columns); + dim3 const dim_grid(frags.size().second, frags.size().first); get_dictionary_indices_kernel - <<>>(chunks, num_rows); + <<>>(chunks, frags, num_rows); } } // namespace gpu } // namespace parquet diff --git a/cpp/src/io/parquet/parquet_gpu.hpp b/cpp/src/io/parquet/parquet_gpu.hpp index 8131a1d301e..3692aa95674 100644 --- a/cpp/src/io/parquet/parquet_gpu.hpp +++ b/cpp/src/io/parquet/parquet_gpu.hpp @@ -257,6 +257,8 @@ struct partition_info { constexpr int max_page_fragment_size = 5000; //!< Max number of rows in a page fragment +struct EncColumnChunk; + /** * @brief Struct describing an encoder page fragment */ @@ -267,11 +269,10 @@ struct PageFragment { uint32_t start_value_idx; uint32_t num_leaf_values; //!< Number of leaf values in fragment. Does not include nulls at //!< non-leaf level - - // Add a start_row member because fragments no longer 5000 rows each - size_type start_row; //!< First row in fragment - uint16_t num_rows; //!< Number of rows in fragment - uint16_t num_dict_vals; //!< Number of unique dictionary entries + size_type start_row; //!< First row in fragment + uint16_t num_rows; //!< Number of rows in fragment + uint16_t num_dict_vals; //!< Number of unique dictionary entries + EncColumnChunk* chunk; //!< The chunk that this fragment belongs to }; /// Size of hash used for building dictionaries @@ -540,6 +541,7 @@ void collect_map_entries(device_span chunks, rmm::cuda_stream_vi * @param stream CUDA stream to use */ void get_dictionary_indices(cudf::detail::device_2dspan chunks, + cudf::detail::device_2dspan frags, size_type num_rows, rmm::cuda_stream_view stream); diff --git a/cpp/src/io/parquet/writer_impl.cu b/cpp/src/io/parquet/writer_impl.cu index 6f4a067dd37..5ac758d8860 100644 --- a/cpp/src/io/parquet/writer_impl.cu +++ b/cpp/src/io/parquet/writer_impl.cu @@ -900,7 +900,7 @@ auto build_chunk_dictionaries(hostdevice_2dvector& chunks, } chunks.host_to_device(stream); gpu::collect_map_entries(chunks.device_view().flat_view(), stream); - gpu::get_dictionary_indices(chunks.device_view(), num_rows, stream); + gpu::get_dictionary_indices(chunks.device_view(), frags, num_rows, stream); return std::make_pair(std::move(dict_data), std::move(dict_index)); } @@ -1342,6 +1342,9 @@ void writer::impl::write(table_view const& table, auto chunk_fragments = fragments[c].subspan(f, fragments_in_chunk); // In fragment struct, add a pointer to the chunk it belongs to // In each fragment in chunk_fragments, update the chunk pointer here. + for (auto& frag : chunk_fragments) { + frag.chunk = &chunks.device_view()[r + first_rg_in_part[p]][c]; + } ck->num_values = std::accumulate( chunk_fragments.begin(), chunk_fragments.end(), 0, [](uint32_t l, auto r) { return l + r.num_values; @@ -1365,18 +1368,19 @@ void writer::impl::write(table_view const& table, } // Pass fragments hd_vec to build_chunk_dictionaries - // auto dict_info_owner = build_chunk_dictionaries(chunks, col_desc, fragments, num_rows, stream); - // for (size_t p = 0; partitions.size(); p++) { - // for (int rg = 0; rg < num_rg_in_part[p]; rg++) { - // size_t global_rg = global_rowgroup_base[p] + rg; - // for (int col = 0; col < num_columns; col++) { - // if (chunks.host_view()[rg][col].use_dictionary) { - // md->files[p].row_groups[global_rg].columns[col].meta_data.encodings.push_back( - // Encoding::PLAIN_DICTIONARY); - // } - // } - // } - // } + fragments.host_to_device(stream); + auto dict_info_owner = build_chunk_dictionaries(chunks, col_desc, fragments, num_rows, stream); + for (size_t p = 0; p < partitions.size(); p++) { + for (int rg = 0; rg < num_rg_in_part[p]; rg++) { + size_t global_rg = global_rowgroup_base[p] + rg; + for (int col = 0; col < num_columns; col++) { + if (chunks.host_view()[rg][col].use_dictionary) { + md->files[p].row_groups[global_rg].columns[col].meta_data.encodings.push_back( + Encoding::PLAIN_DICTIONARY); + } + } + } + } // Build chunk dictionaries and count pages if (num_chunks != 0) { init_page_sizes(chunks, col_desc, num_columns); } diff --git a/cpp/tests/io/parquet_test.cpp b/cpp/tests/io/parquet_test.cpp index 16f3c2d8cbb..5c055d063c7 100644 --- a/cpp/tests/io/parquet_test.cpp +++ b/cpp/tests/io/parquet_test.cpp @@ -2116,13 +2116,14 @@ TEST_F(ParquetWriterStressTest, LargeTableGoodCompression) // exercises multiple rowgroups srand(31337); - auto expected = create_compressible_fixed_table(16, 4 * 1024 * 1024, 10, false); + auto expected = create_compressible_fixed_table(16, 4 * 1024 * 1024, 1000, false); // write out using the custom sink (which uses device writes) cudf_io::parquet_writer_options args = cudf_io::parquet_writer_options::builder( cudf_io::sink_info(std::vector{"first.parquet", "second.parquet"}), *expected) - .partitions({{10, 256 * 1024}, {256 * 1024 + 7, 1024 * 1024}}); + .partitions({{10, 20 * 1024}, {20 * 1024 + 7, 30 * 1024}}) + .compression(cudf_io::compression_type::NONE); cudf_io::write_parquet(args); // cudf_io::parquet_reader_options custom_args = From b2b44a64371c266e6bdce4ba637b5dd2582e0ce1 Mon Sep 17 00:00:00 2001 From: Devavret Makkar Date: Fri, 26 Nov 2021 01:20:24 +0530 Subject: [PATCH 06/71] API cleanups --- cpp/include/cudf/io/detail/parquet.hpp | 6 ++++-- cpp/include/cudf/io/parquet.hpp | 10 ++++++---- cpp/src/io/functions.cpp | 9 +++++---- cpp/src/io/parquet/writer_impl.cu | 1 - 4 files changed, 15 insertions(+), 11 deletions(-) diff --git a/cpp/include/cudf/io/detail/parquet.hpp b/cpp/include/cudf/io/detail/parquet.hpp index 24cc4bc3294..59fd413d1cb 100644 --- a/cpp/include/cudf/io/detail/parquet.hpp +++ b/cpp/include/cudf/io/detail/parquet.hpp @@ -91,7 +91,7 @@ class writer { /** * @brief Constructor for output to a file. * - * @param sink The data sink to write the data to + * @param sinks The data sinks to write the data to * @param options Settings for controlling writing behavior * @param mode Option to write at once or in chunks * @param stream CUDA stream used for device memory operations and kernel launches @@ -106,7 +106,7 @@ class writer { /** * @brief Constructor for writer to handle chunked parquet options. * - * @param sink The data sink to write the data to + * @param sinks The data sinks to write the data to * @param options Settings for controlling writing behavior for chunked writer * @param mode Option to write at once or in chunks * @param stream CUDA stream used for device memory operations and kernel launches @@ -129,6 +129,8 @@ class writer { * @brief Writes a single subtable as part of a larger parquet file/table write. * * @param[in] table The table information to be written + * @param[in] partitions Optional partitions to divide the table into. If specified, must be same + * size as number of sinks. */ void write(table_view const& table, std::vector> const& partitions = {}); diff --git a/cpp/include/cudf/io/parquet.hpp b/cpp/include/cudf/io/parquet.hpp index 7d0715883c3..5cbb9ae9538 100644 --- a/cpp/include/cudf/io/parquet.hpp +++ b/cpp/include/cudf/io/parquet.hpp @@ -394,7 +394,7 @@ class parquet_writer_options { statistics_freq _stats_level = statistics_freq::STATISTICS_ROWGROUP; // Sets of columns to output table_view _table; - // Partitions described in a {start_row, num_rows} pairs + // Partitions described as {start_row, num_rows} pairs std::vector> _partitions; // Optional associated metadata table_input_metadata const* _metadata = nullptr; @@ -497,9 +497,10 @@ class parquet_writer_options { auto get_row_group_size_rows() const { return _row_group_size_rows; } /** - * @brief Sets metadata. + * @brief Sets partitions. * - * @param partitions Associated metadata. + * @param partitions Partitions of input table in {start_row, num_rows} pairs. If specified, must + * be same size as number of sinks in sink_info */ void set_partitions(std::vector> const& partitions) { @@ -593,7 +594,8 @@ class parquet_writer_options_builder { /** * @brief Sets partitions in parquet_writer_options. * - * @param partitions Partitions of input table in {start_row, num_rows} pairs. + * @param partitions Partitions of input table in {start_row, num_rows} pairs. If specified, must + * be same size as number of sinks in sink_info * @return this for chaining. */ parquet_writer_options_builder& partitions( diff --git a/cpp/src/io/functions.cpp b/cpp/src/io/functions.cpp index a755c2c937c..e0259a28842 100644 --- a/cpp/src/io/functions.cpp +++ b/cpp/src/io/functions.cpp @@ -224,8 +224,8 @@ void write_csv(csv_writer_options const& options, rmm::mr::device_memory_resourc { using namespace cudf::io::detail; - // TODO: protect against multiple sinks until we support it in CSV auto sinks = make_datasinks(options.get_sink()); + CUDF_EXPECTS(sinks.size() == 1, "Multiple sinks not supported for CSV writing"); return csv::write_csv( // sinks[0].get(), @@ -357,8 +357,9 @@ void write_orc(orc_writer_options const& options, rmm::mr::device_memory_resourc CUDF_FUNC_RANGE(); - // TODO: protect against multiple sinks until we support it in ORC - auto sinks = make_datasinks(options.get_sink()); + auto sinks = make_datasinks(options.get_sink()); + CUDF_EXPECTS(sinks.size() == 1, "Multiple sinks not supported for ORC writing"); + auto writer = std::make_unique( std::move(sinks[0]), options, io_detail::SingleWriteMode::YES, rmm::cuda_stream_default, mr); @@ -373,8 +374,8 @@ orc_chunked_writer::orc_chunked_writer(chunked_orc_writer_options const& options { namespace io_detail = cudf::io::detail; - // TODO: protect against multiple sinks until we support it in ORC auto sinks = make_datasinks(options.get_sink()); + CUDF_EXPECTS(sinks.size() == 1, "Multiple sinks not supported for ORC writing"); writer = std::make_unique( std::move(sinks[0]), options, io_detail::SingleWriteMode::NO, rmm::cuda_stream_default, mr); diff --git a/cpp/src/io/parquet/writer_impl.cu b/cpp/src/io/parquet/writer_impl.cu index 5ac758d8860..ac0c30422b2 100644 --- a/cpp/src/io/parquet/writer_impl.cu +++ b/cpp/src/io/parquet/writer_impl.cu @@ -79,7 +79,6 @@ parquet::Compression to_parquet_compression(compression_type compression) } // namespace -// TODO: now I think partition is a better name than file struct per_file_metadata { per_file_metadata(int num_files) : files(num_files) {} From 0b6d33f3a20311ca3b3b8c78a7b14e0bbf73009b Mon Sep 17 00:00:00 2001 From: Devavret Makkar Date: Fri, 26 Nov 2021 19:38:46 +0530 Subject: [PATCH 07/71] Add a gtest and fix other tests by handling no partition case --- cpp/src/io/parquet/writer_impl.cu | 5 ++--- cpp/tests/io/parquet_test.cpp | 30 ++++++++++++++++++++++++++++++ 2 files changed, 32 insertions(+), 3 deletions(-) diff --git a/cpp/src/io/parquet/writer_impl.cu b/cpp/src/io/parquet/writer_impl.cu index ac0c30422b2..55213155662 100644 --- a/cpp/src/io/parquet/writer_impl.cu +++ b/cpp/src/io/parquet/writer_impl.cu @@ -1080,7 +1080,6 @@ writer::impl::impl(std::vector> sinks, single_write_mode(mode == SingleWriteMode::YES), out_sink_(std::move(sinks)) { - // out_sink_.push_back(std::move(sink)); if (options.get_metadata()) { table_meta = std::make_unique(*options.get_metadata()); } @@ -1102,7 +1101,6 @@ writer::impl::impl(std::vector> sinks, single_write_mode(mode == SingleWriteMode::YES), out_sink_(std::move(sinks)) { - // out_sink_.push_back(std::move(sink)); if (options.get_metadata()) { table_meta = std::make_unique(*options.get_metadata()); } @@ -1127,8 +1125,9 @@ void writer::impl::write(table_view const& table, std::vector> partitions) { CUDF_EXPECTS(not closed, "Data has already been flushed to out and closed"); - // Verify that partitions don't overlap + if (partitions.empty()) { partitions.push_back({0, table.num_rows()}); } + // TODO: remove size_type num_rows = table.num_rows(); if (not table_meta) { table_meta = std::make_unique(table); } diff --git a/cpp/tests/io/parquet_test.cpp b/cpp/tests/io/parquet_test.cpp index 5c055d063c7..fe2f4f97532 100644 --- a/cpp/tests/io/parquet_test.cpp +++ b/cpp/tests/io/parquet_test.cpp @@ -1170,6 +1170,36 @@ TEST_F(ParquetWriterTest, DeviceWriteLargeishFile) auto custom_tbl = cudf_io::read_parquet(custom_args); CUDF_TEST_EXPECT_TABLES_EQUAL(custom_tbl.tbl->view(), expected->view()); } + +TEST_F(ParquetWriterTest, PartitionedWrite) +{ + auto source = create_compressible_fixed_table(16, 4 * 1024 * 1024, 1000, false); + + auto filepath1 = temp_env->get_temp_filepath("PartitionedWrite1.parquet"); + auto filepath2 = temp_env->get_temp_filepath("PartitionedWrite2.parquet"); + + auto partition1 = std::make_pair(10, 1024 * 1024); + auto partition2 = std::make_pair(20 * 1024 + 7, 3 * 1024 * 1024); + + auto expected1 = cudf::slice(*source, {partition1.first, partition1.first + partition1.second}); + auto expected2 = cudf::slice(*source, {partition2.first, partition2.first + partition2.second}); + + cudf_io::parquet_writer_options args = + cudf_io::parquet_writer_options::builder( + cudf_io::sink_info(std::vector{filepath1, filepath2}), *source) + .partitions({partition1, partition2}) + .compression(cudf_io::compression_type::NONE); + cudf_io::write_parquet(args); + + auto result1 = cudf_io::read_parquet( + cudf_io::parquet_reader_options::builder(cudf_io::source_info(filepath1))); + CUDF_TEST_EXPECT_TABLES_EQUAL(expected1, result1.tbl->view()); + + auto result2 = cudf_io::read_parquet( + cudf_io::parquet_reader_options::builder(cudf_io::source_info(filepath2))); + CUDF_TEST_EXPECT_TABLES_EQUAL(expected2, result2.tbl->view()); +} + template std::string create_parquet_file(int num_cols) { From 2beed73f8d7a3401f72e554a4b391dbb57f00874 Mon Sep 17 00:00:00 2001 From: Devavret Makkar Date: Sat, 27 Nov 2021 01:00:34 +0530 Subject: [PATCH 08/71] Add a guard to protect from an exception being thrown in impl dtor when a previour write(table) call failed --- cpp/src/io/parquet/writer_impl.cu | 3 +++ cpp/src/io/parquet/writer_impl.hpp | 2 ++ 2 files changed, 5 insertions(+) diff --git a/cpp/src/io/parquet/writer_impl.cu b/cpp/src/io/parquet/writer_impl.cu index 55213155662..b294ebc485b 100644 --- a/cpp/src/io/parquet/writer_impl.cu +++ b/cpp/src/io/parquet/writer_impl.cu @@ -1124,6 +1124,7 @@ void writer::impl::init_state() void writer::impl::write(table_view const& table, std::vector> partitions) { + last_write_successful = false; CUDF_EXPECTS(not closed, "Data has already been flushed to out and closed"); if (partitions.empty()) { partitions.push_back({0, table.num_rows()}); } @@ -1595,6 +1596,7 @@ void writer::impl::write(table_view const& table, task.wait(); } } + last_write_successful = true; } std::unique_ptr> writer::impl::close( @@ -1602,6 +1604,7 @@ std::unique_ptr> writer::impl::close( { if (closed) { return nullptr; } closed = true; + if (not last_write_successful) { return nullptr; } for (size_t p = 0; p < out_sink_.size(); p++) { std::vector buffer; CompactProtocolWriter cpw(&buffer); diff --git a/cpp/src/io/parquet/writer_impl.hpp b/cpp/src/io/parquet/writer_impl.hpp index 838cb373ddc..add90d389e4 100644 --- a/cpp/src/io/parquet/writer_impl.hpp +++ b/cpp/src/io/parquet/writer_impl.hpp @@ -216,6 +216,8 @@ class writer::impl { std::unique_ptr table_meta; // to track if the output has been written to sink bool closed = false; + // To track if the last write(table) call completed successfully + bool last_write_successful = false; // current write position for rowgroups/chunks std::vector current_chunk_offset; // special parameter only used by detail::write() to indicate that we are guaranteeing From 4e21e99b21741e9c3bb254be3105f4ff8a1714fb Mon Sep 17 00:00:00 2001 From: Devavret Makkar Date: Tue, 30 Nov 2021 01:49:43 +0530 Subject: [PATCH 09/71] Add per-sink user_data in table_input_metadata --- cpp/include/cudf/io/types.hpp | 27 +++++++++++++++++++++++--- cpp/src/io/functions.cpp | 12 ++++++++++++ cpp/src/io/orc/writer_impl.cu | 15 ++++++++------ cpp/src/io/parquet/writer_impl.cu | 16 ++++++++------- cpp/tests/io/parquet_test.cpp | 2 +- python/cudf/cudf/_lib/cpp/io/types.pxd | 2 +- python/cudf/cudf/_lib/orc.pyx | 2 +- python/cudf/cudf/_lib/parquet.pyx | 15 ++++++-------- 8 files changed, 63 insertions(+), 28 deletions(-) diff --git a/cpp/include/cudf/io/types.hpp b/cpp/include/cudf/io/types.hpp index ee4cdc2d004..87c5e999776 100644 --- a/cpp/include/cudf/io/types.hpp +++ b/cpp/include/cudf/io/types.hpp @@ -385,12 +385,33 @@ class table_input_metadata { * The constructed table_input_metadata has the same structure as the passed table_view * * @param table The table_view to construct metadata for - * @param user_data Optional Additional metadata to encode, as key-value pairs */ - table_input_metadata(table_view const& table, std::map user_data = {}); + table_input_metadata(table_view const& table); + + /** + * @brief Construct a new table_input_metadata from a table_view. + * + * The constructed table_input_metadata has the same structure as the passed table_view + * + * @param table The table_view to construct metadata for + * @param user_data Additional metadata to encode, as key-value pairs + */ + table_input_metadata(table_view const& table, std::map user_data); + + /** + * @brief Construct a new table_input_metadata from a table_view. + * + * The constructed table_input_metadata has the same structure as the passed table_view + * + * @param table The table_view to construct metadata for + * @param user_data Additional metadata to encode, as key-value pairs. One per output file + */ + table_input_metadata(table_view const& table, + std::vector> user_data); std::vector column_metadata; - std::map user_data; //!< Format-dependent metadata as key-values pairs + std::vector> + user_data; //!< Format-dependent metadata as key-values pairs. One per output file }; } // namespace io diff --git a/cpp/src/io/functions.cpp b/cpp/src/io/functions.cpp index e0259a28842..bfe146bc9a5 100644 --- a/cpp/src/io/functions.cpp +++ b/cpp/src/io/functions.cpp @@ -428,8 +428,20 @@ std::unique_ptr> merge_row_group_metadata( return detail_parquet::writer::merge_row_group_metadata(metadata_list); } +table_input_metadata::table_input_metadata(table_view const& table) + : table_input_metadata(table, std::map{}) +{ +} + table_input_metadata::table_input_metadata(table_view const& table, std::map user_data) + : table_input_metadata(table, + std::vector>{std::move(user_data)}) +{ +} + +table_input_metadata::table_input_metadata( + table_view const& table, std::vector> user_data) : user_data{std::move(user_data)} { // Create a metadata hierarchy using `table` diff --git a/cpp/src/io/orc/writer_impl.cu b/cpp/src/io/orc/writer_impl.cu index 25c4bd65c8f..e3b18b80f2e 100644 --- a/cpp/src/io/orc/writer_impl.cu +++ b/cpp/src/io/orc/writer_impl.cu @@ -2053,12 +2053,15 @@ void writer::impl::close() PostScript ps; ff.contentLength = out_sink_->bytes_written(); - std::transform(table_meta->user_data.begin(), - table_meta->user_data.end(), - std::back_inserter(ff.metadata), - [&](auto const& udata) { - return UserMetadataItem{udata.first, udata.second}; - }); + if (not table_meta->user_data.empty()) { + // ORC writer currently does not support multiple file writing + std::transform(table_meta->user_data[0].begin(), + table_meta->user_data[0].end(), + std::back_inserter(ff.metadata), + [&](auto const& udata) { + return UserMetadataItem{udata.first, udata.second}; + }); + } // Write statistics metadata if (md.stripeStats.size() != 0) { diff --git a/cpp/src/io/parquet/writer_impl.cu b/cpp/src/io/parquet/writer_impl.cu index b294ebc485b..951c270a47f 100644 --- a/cpp/src/io/parquet/writer_impl.cu +++ b/cpp/src/io/parquet/writer_impl.cu @@ -1176,13 +1176,15 @@ void writer::impl::write(table_view const& table, // Think about how this will be passed. Currently it is passed in table_input_metadata which is // only passed once as part of args to writer ctor. Now this would need to be passed per sink. - // But we only need them once. Just like the list of sinks. Maybe we can - // std::transform(table_meta->user_data.begin(), - // table_meta->user_data.end(), - // std::back_inserter(md.key_value_metadata), - // [](auto const& kv) { - // return KeyValue{kv.first, kv.second}; - // }); + // But we only need them once. Ask in review + for (size_t p = 0; p < table_meta->user_data.size(); ++p) { + std::transform(table_meta->user_data[p].begin(), + table_meta->user_data[p].end(), + std::back_inserter(md->files[p].key_value_metadata), + [](auto const& kv) { + return KeyValue{kv.first, kv.second}; + }); + } md->schema = this_table_schema; } else { // verify the user isn't passing mismatched tables diff --git a/cpp/tests/io/parquet_test.cpp b/cpp/tests/io/parquet_test.cpp index fe2f4f97532..bcf2214a70e 100644 --- a/cpp/tests/io/parquet_test.cpp +++ b/cpp/tests/io/parquet_test.cpp @@ -834,7 +834,7 @@ TEST_F(ParquetWriterTest, MultiIndex) expected_metadata.column_metadata[2].set_name("int32s"); expected_metadata.column_metadata[3].set_name("floats"); expected_metadata.column_metadata[4].set_name("doubles"); - expected_metadata.user_data.insert( + expected_metadata.user_data[0].insert( {"pandas", "\"index_columns\": [\"int8s\", \"int16s\"], \"column1\": [\"int32s\"]"}); auto filepath = temp_env->get_temp_filepath("MultiIndex.parquet"); diff --git a/python/cudf/cudf/_lib/cpp/io/types.pxd b/python/cudf/cudf/_lib/cpp/io/types.pxd index 6b68902d22f..1db9a0e68f5 100644 --- a/python/cudf/cudf/_lib/cpp/io/types.pxd +++ b/python/cudf/cudf/_lib/cpp/io/types.pxd @@ -76,7 +76,7 @@ cdef extern from "cudf/io/types.hpp" \ ) except + vector[column_in_metadata] column_metadata - map[string, string] user_data + vector[map[string, string]] user_data cdef cppclass host_buffer: const char* data diff --git a/python/cudf/cudf/_lib/orc.pyx b/python/cudf/cudf/_lib/orc.pyx index 1281aa172b4..3ce9d4f0bcc 100644 --- a/python/cudf/cudf/_lib/orc.pyx +++ b/python/cudf/cudf/_lib/orc.pyx @@ -340,7 +340,7 @@ cdef class ORCWriter: ) pandas_metadata = generate_pandas_metadata(table, self.index) - self.tbl_meta.get().user_data[str.encode("pandas")] = \ + self.tbl_meta.get().user_data[0][str.encode("pandas")] = \ str.encode(pandas_metadata) cdef chunked_orc_writer_options args diff --git a/python/cudf/cudf/_lib/parquet.pyx b/python/cudf/cudf/_lib/parquet.pyx index d17184685fa..c77144ead45 100644 --- a/python/cudf/cudf/_lib/parquet.pyx +++ b/python/cudf/cudf/_lib/parquet.pyx @@ -301,11 +301,14 @@ cpdef write_parquet( cdef unique_ptr[cudf_io_types.data_sink] _data_sink cdef cudf_io_types.sink_info sink = make_sink_info(path, _data_sink) + pandas_metadata = generate_pandas_metadata(table, index) + user_data[str.encode("pandas")] = str.encode(pandas_metadata) + if index is True or ( index is None and not isinstance(table._index, cudf.RangeIndex) ): tv = table_view_from_table(table) - tbl_meta = make_unique[table_input_metadata](tv) + tbl_meta = make_unique[table_input_metadata](tv, move(user_data)) for level, idx_name in enumerate(table._index.names): tbl_meta.get().column_metadata[level].set_name( str.encode( @@ -315,7 +318,7 @@ cpdef write_parquet( num_index_cols_meta = len(table._index.names) else: tv = table_view_from_table(table, ignore_index=True) - tbl_meta = make_unique[table_input_metadata](tv) + tbl_meta = make_unique[table_input_metadata](tv, move(user_data)) num_index_cols_meta = 0 for i, name in enumerate(table._column_names, num_index_cols_meta): @@ -327,12 +330,6 @@ cpdef write_parquet( table[name]._column, tbl_meta.get().column_metadata[i] ) - pandas_metadata = generate_pandas_metadata(table, index) - user_data[str.encode("pandas")] = str.encode(pandas_metadata) - - # Set the table_metadata - tbl_meta.get().user_data = user_data - cdef cudf_io_types.compression_type comp_type = _get_comp_type(compression) cdef cudf_io_types.statistics_freq stat_freq = _get_stat_freq(statistics) @@ -471,7 +468,7 @@ cdef class ParquetWriter: ) pandas_metadata = generate_pandas_metadata(table, self.index) - self.tbl_meta.get().user_data[str.encode("pandas")] = \ + self.tbl_meta.get().user_data[0][str.encode("pandas")] = \ str.encode(pandas_metadata) cdef chunked_parquet_writer_options args From e0d1f3331df7161324cb855e62753f05ff115323 Mon Sep 17 00:00:00 2001 From: Devavret Makkar Date: Wed, 1 Dec 2021 04:41:37 +0530 Subject: [PATCH 10/71] Cleanups in dict code and replace index translating while LIST loop with function --- cpp/src/io/parquet/chunk_dict.cu | 119 ++++++----------------------- cpp/src/io/parquet/page_enc.cu | 60 +++------------ cpp/src/io/parquet/parquet_gpu.hpp | 21 +++++ 3 files changed, 54 insertions(+), 146 deletions(-) diff --git a/cpp/src/io/parquet/chunk_dict.cu b/cpp/src/io/parquet/chunk_dict.cu index f73af258109..c00aca2ec79 100644 --- a/cpp/src/io/parquet/chunk_dict.cu +++ b/cpp/src/io/parquet/chunk_dict.cu @@ -103,71 +103,34 @@ __global__ void __launch_bounds__(block_size, 1) auto t = threadIdx.x; auto frag = frags[col_idx][block_x]; auto chunk = frag.chunk; + auto col = chunk->col_desc; - // auto start_row = - // block_x * - // max_page_fragment_size; // This is fragment size. all chunks are multiple of these many - // rows. - // size_type end_row = min(start_row + max_page_fragment_size, num_rows); size_type start_row = frag.start_row; size_type end_row = frag.start_row + frag.num_rows; - // TODO: s_chunk maybe not needed - __shared__ EncColumnChunk* s_chunk; - __shared__ parquet_column_device_view s_col; __shared__ size_type s_start_value_idx; __shared__ size_type s_num_values; - // TODO: cleanup. No need to get chunk like this - if (t == 0) { - // // Find the chunk this block is a part of - // size_type num_rowgroups = chunks.size().first; - // size_type rg_idx = 0; - // while (rg_idx < num_rowgroups) { - // if (auto ck = chunks[rg_idx][col_idx]; - // start_row >= ck.start_row and start_row < ck.start_row + ck.num_rows) { - // break; - // } - // ++rg_idx; - // } - // s_chunk = &chunks[rg_idx][col_idx]; - s_chunk = chunk; - s_col = *(s_chunk->col_desc); - } - __syncthreads(); - if (not s_chunk->use_dictionary) { return; } + if (not chunk->use_dictionary) { return; } if (t == 0) { // Find the bounds of values in leaf column to be inserted into the map for current chunk - auto col = *(s_col.parent_column); - auto start_value_idx = start_row; - auto end_value_idx = end_row; - while (col.type().id() == type_id::LIST or col.type().id() == type_id::STRUCT) { - if (col.type().id() == type_id::STRUCT) { - start_value_idx += col.offset(); - end_value_idx += col.offset(); - col = col.child(0); - } else { - auto offset_col = col.child(lists_column_view::offsets_column_index); - start_value_idx = offset_col.element(start_value_idx + col.offset()); - end_value_idx = offset_col.element(end_value_idx + col.offset()); - col = col.child(lists_column_view::child_column_index); - } - } - s_start_value_idx = start_value_idx; - s_num_values = end_value_idx - start_value_idx; + auto cudf_col = *(col->parent_column); + s_start_value_idx = row_to_value_idx(start_row, cudf_col); + auto end_value_idx = row_to_value_idx(end_row, cudf_col); + s_num_values = end_value_idx - s_start_value_idx; } __syncthreads(); - column_device_view const& data_col = *s_col.leaf_column; + column_device_view const& data_col = *col->leaf_column; using block_reduce = cub::BlockReduce; __shared__ typename block_reduce::TempStorage reduce_storage; // Make a view of the hash map auto hash_map_mutable = map_type::device_mutable_view( - s_chunk->dict_map_slots, s_chunk->dict_map_size, KEY_SENTINEL, VALUE_SENTINEL); + chunk->dict_map_slots, chunk->dict_map_size, KEY_SENTINEL, VALUE_SENTINEL); auto hash_map = map_type::device_view( - s_chunk->dict_map_slots, s_chunk->dict_map_size, KEY_SENTINEL, VALUE_SENTINEL); + chunk->dict_map_slots, chunk->dict_map_size, KEY_SENTINEL, VALUE_SENTINEL); __shared__ int total_num_dict_entries; for (size_type i = 0; i < s_num_values; i += block_size) { @@ -186,7 +149,7 @@ __global__ void __launch_bounds__(block_size, 1) type_dispatcher(data_col.type(), map_insert_fn{hash_map_mutable}, data_col, val_idx); uniq_elem_size = [&]() -> size_type { if (not is_unique) { return 0; } - switch (s_col.physical_type) { + switch (col->physical_type) { case Type::INT32: return 4; case Type::INT64: return 8; case Type::INT96: return 12; @@ -209,9 +172,9 @@ __global__ void __launch_bounds__(block_size, 1) __syncthreads(); auto uniq_data_size = block_reduce(reduce_storage).Sum(uniq_elem_size); if (t == 0) { - total_num_dict_entries = atomicAdd(&s_chunk->num_dict_entries, num_unique); + total_num_dict_entries = atomicAdd(&chunk->num_dict_entries, num_unique); total_num_dict_entries += num_unique; - atomicAdd(&s_chunk->uniq_data_size, uniq_data_size); + atomicAdd(&chunk->uniq_data_size, uniq_data_size); } __syncthreads(); @@ -262,66 +225,32 @@ __global__ void __launch_bounds__(block_size, 1) auto block_x = blockIdx.x; auto t = threadIdx.x; auto frag = frags[col_idx][block_x]; - // auto chunk = frag.chunk; + auto chunk = frag.chunk; + auto col = chunk->col_desc; - // size_type start_row = block_x * max_page_fragment_size; - // size_type end_row = min(start_row + max_page_fragment_size, num_rows); size_type start_row = frag.start_row; size_type end_row = frag.start_row + frag.num_rows; - __shared__ EncColumnChunk s_chunk; - __shared__ parquet_column_device_view s_col; __shared__ size_type s_start_value_idx; __shared__ size_type s_ck_start_val_idx; __shared__ size_type s_num_values; if (t == 0) { - // // Find the chunk this block is a part of - // size_type num_rowgroups = chunks.size().first; - // size_type rg_idx = 0; - // while (rg_idx < num_rowgroups) { - // if (auto ck = chunks[rg_idx][col_idx]; - // start_row >= ck.start_row and start_row < ck.start_row + ck.num_rows) { - // break; - // } - // ++rg_idx; - // } - // s_chunk = chunks[rg_idx][col_idx]; - s_chunk = *frag.chunk; - s_col = *(s_chunk.col_desc); - - // Find the bounds of values in leaf column to be inserted into the map for current chunk - - auto col = *(s_col.parent_column); - auto start_value_idx = start_row; - auto end_value_idx = end_row; - auto chunk_start_val_idx = s_chunk.start_row; - while (col.type().id() == type_id::LIST or col.type().id() == type_id::STRUCT) { - if (col.type().id() == type_id::STRUCT) { - start_value_idx += col.offset(); - chunk_start_val_idx += col.offset(); - end_value_idx += col.offset(); - col = col.child(0); - } else { - auto offset_col = col.child(lists_column_view::offsets_column_index); - start_value_idx = offset_col.element(start_value_idx + col.offset()); - chunk_start_val_idx = offset_col.element(chunk_start_val_idx + col.offset()); - end_value_idx = offset_col.element(end_value_idx + col.offset()); - col = col.child(lists_column_view::child_column_index); - } - } - s_start_value_idx = start_value_idx; - s_ck_start_val_idx = chunk_start_val_idx; - s_num_values = end_value_idx - start_value_idx; + // Find the bounds of values in leaf column to be searched in the map for current chunk + auto cudf_col = *(col->parent_column); + s_start_value_idx = row_to_value_idx(start_row, cudf_col); + s_ck_start_val_idx = row_to_value_idx(chunk->start_row, cudf_col); + auto end_value_idx = row_to_value_idx(end_row, cudf_col); + s_num_values = end_value_idx - s_start_value_idx; } __syncthreads(); - if (not s_chunk.use_dictionary) { return; } + if (not chunk->use_dictionary) { return; } - column_device_view const& data_col = *s_col.leaf_column; + column_device_view const& data_col = *col->leaf_column; auto map = map_type::device_view( - s_chunk.dict_map_slots, s_chunk.dict_map_size, KEY_SENTINEL, VALUE_SENTINEL); + chunk->dict_map_slots, chunk->dict_map_size, KEY_SENTINEL, VALUE_SENTINEL); for (size_t i = 0; i < s_num_values; i += block_size) { if (t + i < s_num_values) { @@ -336,7 +265,7 @@ __global__ void __launch_bounds__(block_size, 1) if (found_slot != map.end()) { // No need for atomic as this is not going to be modified by any other thread auto* val_ptr = reinterpret_cast(&found_slot->second); - s_chunk.dict_index[val_idx - s_ck_start_val_idx] = *val_ptr; + chunk->dict_index[val_idx - s_ck_start_val_idx] = *val_ptr; } } } diff --git a/cpp/src/io/parquet/page_enc.cu b/cpp/src/io/parquet/page_enc.cu index 810dd1a45c0..0b2b3a7eefd 100644 --- a/cpp/src/io/parquet/page_enc.cu +++ b/cpp/src/io/parquet/page_enc.cu @@ -146,34 +146,11 @@ __global__ void __launch_bounds__(block_size) s->frag.fragment_data_size = 0; s->frag.dict_data_size = 0; - // To use num_vals instead of num_rows, we need to calculate num_vals on the fly. - // For list>, values between i and i+50 can be calculated by - // off_11 = off[i], off_12 = off[i+50] - // off_21 = child.off[off_11], off_22 = child.off[off_12] - // etc... - size_type end_value_idx = s->frag.start_row + s->frag.num_rows; - if (s->col.parent_column == nullptr) { - s->start_value_idx = s->frag.start_row; - } else { - auto col = *(s->col.parent_column); - auto current_start_value_idx = s->frag.start_row; - while (col.type().id() == type_id::LIST or col.type().id() == type_id::STRUCT) { - if (col.type().id() == type_id::STRUCT) { - current_start_value_idx += col.offset(); - end_value_idx += col.offset(); - col = col.child(0); - } else { - auto offset_col = col.child(lists_column_view::offsets_column_index); - current_start_value_idx = - offset_col.element(current_start_value_idx + col.offset()); - end_value_idx = offset_col.element(end_value_idx + col.offset()); - col = col.child(lists_column_view::child_column_index); - } - } - s->start_value_idx = current_start_value_idx; - } - s->frag.start_value_idx = s->start_value_idx; - s->frag.num_leaf_values = end_value_idx - s->start_value_idx; + size_type end_row = s->frag.start_row + s->frag.num_rows; + auto col = *(s->col.parent_column); + s->frag.start_value_idx = row_to_value_idx(s->frag.start_row, col); + size_type end_value_idx = row_to_value_idx(end_row, col); + s->frag.num_leaf_values = end_value_idx - s->frag.start_value_idx; if (s->col.level_offsets != nullptr) { // For nested schemas, the number of values in a fragment is not directly related to the @@ -194,7 +171,7 @@ __global__ void __launch_bounds__(block_size) __syncthreads(); size_type nvals = s->frag.num_leaf_values; - size_type start_value_idx = s->start_value_idx; + size_type start_value_idx = s->frag.start_value_idx; for (uint32_t i = 0; i < nvals; i += block_size) { uint32_t val_idx = start_value_idx + i + t; @@ -914,28 +891,9 @@ __global__ void __launch_bounds__(128, 8) dst[0] = dict_bits; s->rle_out = dst + 1; } - s->page_start_val = s->page.start_row; // Dictionary page's start row is chunk's start row - auto chunk_start_val = s->ck.start_row; - if (s->col.parent_column != nullptr) { // TODO: remove this check. parent is now never nullptr - auto col = *(s->col.parent_column); - auto current_page_start_val = s->page_start_val; - // TODO: We do this so much. Add a global function that converts row idx to val idx - while (col.type().id() == type_id::LIST or col.type().id() == type_id::STRUCT) { - if (col.type().id() == type_id::STRUCT) { - current_page_start_val += col.offset(); - chunk_start_val += col.offset(); - col = col.child(0); - } else { - auto offset_col = col.child(lists_column_view::offsets_column_index); - current_page_start_val = - offset_col.element(current_page_start_val + col.offset()); - chunk_start_val = offset_col.element(chunk_start_val + col.offset()); - col = col.child(lists_column_view::child_column_index); - } - } - s->page_start_val = current_page_start_val; - s->chunk_start_val = chunk_start_val; - } + auto col = *(s->col.parent_column); + s->page_start_val = row_to_value_idx(s->page.start_row, col); + s->chunk_start_val = row_to_value_idx(s->ck.start_row, col); } __syncthreads(); for (uint32_t cur_val_idx = 0; cur_val_idx < s->page.num_leaf_values;) { diff --git a/cpp/src/io/parquet/parquet_gpu.hpp b/cpp/src/io/parquet/parquet_gpu.hpp index 3692aa95674..e5498018a4c 100644 --- a/cpp/src/io/parquet/parquet_gpu.hpp +++ b/cpp/src/io/parquet/parquet_gpu.hpp @@ -293,6 +293,27 @@ inline uint32_t __device__ GetDtypeLogicalLen(column_device_view const* col) } } +/** + * @brief Translate the row index of a parent column_device_view into the index of the first value + * in the leaf child. + * Only works in the context of parquet writer where struct columns are previously modified s.t. + * they only have one immediate child. + */ +inline size_type __device__ row_to_value_idx(size_type idx, column_device_view col) +{ + while (col.type().id() == type_id::LIST or col.type().id() == type_id::STRUCT) { + if (col.type().id() == type_id::STRUCT) { + idx += col.offset(); + col = col.child(0); + } else { + auto offset_col = col.child(lists_column_view::offsets_column_index); + idx = offset_col.element(idx + col.offset()); + col = col.child(lists_column_view::child_column_index); + } + } + return idx; +} + /** * @brief Return worst-case compressed size of compressed data given the uncompressed size */ From 54de724756914907115ebdc09fa9cd503f78494b Mon Sep 17 00:00:00 2001 From: Devavret Makkar Date: Wed, 1 Dec 2021 17:49:23 +0530 Subject: [PATCH 11/71] fix the returned metadata blob on close --- cpp/include/cudf/io/detail/parquet.hpp | 3 +- cpp/include/cudf/io/parquet.hpp | 32 +++++++----- cpp/src/io/functions.cpp | 4 +- cpp/src/io/parquet/writer_impl.cu | 65 +++++++++++++++--------- cpp/src/io/parquet/writer_impl.hpp | 4 +- cpp/tests/io/parquet_test.cpp | 2 +- python/cudf/cudf/_lib/cpp/io/parquet.pxd | 12 ++--- python/cudf/cudf/_lib/parquet.pyx | 12 ++--- 8 files changed, 80 insertions(+), 54 deletions(-) diff --git a/cpp/include/cudf/io/detail/parquet.hpp b/cpp/include/cudf/io/detail/parquet.hpp index 59fd413d1cb..3c0f4ada247 100644 --- a/cpp/include/cudf/io/detail/parquet.hpp +++ b/cpp/include/cudf/io/detail/parquet.hpp @@ -143,7 +143,8 @@ class writer { * @return A parquet-compatible blob that contains the data for all rowgroups in the list only if * `column_chunks_file_path` is provided, else null. */ - std::unique_ptr> close(std::string const& column_chunks_file_path = ""); + std::unique_ptr> close( + std::vector const& column_chunks_file_path = {}); /** * @brief Merges multiple metadata blobs returned by write_all into a single metadata blob diff --git a/cpp/include/cudf/io/parquet.hpp b/cpp/include/cudf/io/parquet.hpp index 5cbb9ae9538..7e4406648df 100644 --- a/cpp/include/cudf/io/parquet.hpp +++ b/cpp/include/cudf/io/parquet.hpp @@ -401,8 +401,8 @@ class parquet_writer_options { // Parquet writer can write INT96 or TIMESTAMP_MICROS. Defaults to TIMESTAMP_MICROS. // If true then overrides any per-column setting in _metadata. bool _write_timestamps_as_int96 = false; - // Column chunks file path to be set in the raw output metadata - std::string _column_chunks_file_path; + // Column chunks file paths to be set in the raw output metadata. One per output file + std::vector _column_chunks_file_paths; // Maximum size of each row group (unless smaller than a single page) size_t _row_group_size_bytes = default_row_group_size_bytes; // Maximum number of rows in row group (unless smaller than a single page) @@ -482,9 +482,12 @@ class parquet_writer_options { bool is_enabled_int96_timestamps() const { return _write_timestamps_as_int96; } /** - * @brief Returns Column chunks file path to be set in the raw output metadata. + * @brief Returns Column chunks file paths to be set in the raw output metadata. */ - std::string get_column_chunks_file_path() const { return _column_chunks_file_path; } + std::vector get_column_chunks_file_paths() const + { + return _column_chunks_file_paths; + } /** * @brief Returns maximum row group size, in bytes. @@ -539,11 +542,11 @@ class parquet_writer_options { /** * @brief Sets column chunks file path to be set in the raw output metadata. * - * @param file_path String which indicates file path. + * @param file_paths Vector of Strings which indicates file path. */ - void set_column_chunks_file_path(std::string file_path) + void set_column_chunks_file_paths(std::vector const& file_paths) { - _column_chunks_file_path.assign(file_path); + _column_chunks_file_paths = file_paths; } /** @@ -644,12 +647,13 @@ class parquet_writer_options_builder { /** * @brief Sets column chunks file path to be set in the raw output metadata. * - * @param file_path String which indicates file path. + * @param file_paths Vector of Strings which indicates file path. * @return this for chaining. */ - parquet_writer_options_builder& column_chunks_file_path(std::string file_path) + parquet_writer_options_builder& column_chunks_file_paths( + std::vector const& file_paths) { - options._column_chunks_file_path.assign(file_path); + options._column_chunks_file_paths = file_paths; return *this; } @@ -1030,11 +1034,13 @@ class parquet_chunked_writer { /** * @brief Finishes the chunked/streamed write process. * - * @param[in] column_chunks_file_path Column chunks file path to be set in the raw output metadata + * @param[in] column_chunks_file_paths Column chunks file path to be set in the raw output + * metadata * @return A parquet-compatible blob that contains the data for all rowgroups in the list only if - * `column_chunks_file_path` is provided, else null. + * `column_chunks_file_paths` is provided, else null. */ - std::unique_ptr> close(std::string const& column_chunks_file_path = ""); + std::unique_ptr> close( + std::vector const& column_chunks_file_paths = {}); // Unique pointer to impl writer class std::unique_ptr writer; diff --git a/cpp/src/io/functions.cpp b/cpp/src/io/functions.cpp index bfe146bc9a5..f6ead9084ca 100644 --- a/cpp/src/io/functions.cpp +++ b/cpp/src/io/functions.cpp @@ -472,7 +472,7 @@ std::unique_ptr> write_parquet(parquet_writer_options const writer->write(options.get_table(), options.get_partitions()); - return writer->close(options.get_column_chunks_file_path()); + return writer->close(options.get_column_chunks_file_paths()); } /** @@ -506,7 +506,7 @@ parquet_chunked_writer& parquet_chunked_writer::write( * @copydoc cudf::io::parquet_chunked_writer::close */ std::unique_ptr> parquet_chunked_writer::close( - std::string const& column_chunks_file_path) + std::vector const& column_chunks_file_path) { CUDF_FUNC_RANGE(); return writer->close(column_chunks_file_path); diff --git a/cpp/src/io/parquet/writer_impl.cu b/cpp/src/io/parquet/writer_impl.cu index 951c270a47f..29aebed598b 100644 --- a/cpp/src/io/parquet/writer_impl.cu +++ b/cpp/src/io/parquet/writer_impl.cu @@ -1602,7 +1602,7 @@ void writer::impl::write(table_view const& table, } std::unique_ptr> writer::impl::close( - std::string const& column_chunks_file_path) + std::vector const& column_chunks_file_path) { if (closed) { return nullptr; } closed = true; @@ -1619,26 +1619,44 @@ std::unique_ptr> writer::impl::close( out_sink_[p]->flush(); } - // // Optionally output raw file metadata with the specified column chunk file path - // if (column_chunks_file_path.length() > 0) { - // file_header_s fhdr = {parquet_magic}; - // buffer_.resize(0); - // buffer_.insert(buffer_.end(), - // reinterpret_cast(&fhdr), - // reinterpret_cast(&fhdr) + sizeof(fhdr)); - // for (auto& rowgroup : md.row_groups) { - // for (auto& col : rowgroup.columns) { - // col.file_path = column_chunks_file_path; - // } - // } - // fendr.footer_len = static_cast(cpw.write(md)); - // buffer_.insert(buffer_.end(), - // reinterpret_cast(&fendr), - // reinterpret_cast(&fendr) + sizeof(fendr)); - // return std::make_unique>(std::move(buffer_)); - // } else { - // return {nullptr}; - // } + // Optionally output raw file metadata with the specified column chunk file path + if (column_chunks_file_path.size() > 0) { + CUDF_EXPECTS(column_chunks_file_path.size() == md->files.size(), + "Expected one column chunk path per output file"); + file_header_s fhdr = {parquet_magic}; + std::vector buffer; + CompactProtocolWriter cpw(&buffer); + buffer.insert(buffer.end(), + reinterpret_cast(&fhdr), + reinterpret_cast(&fhdr) + sizeof(fhdr)); + FileMetaData merged_md; + for (size_t p = 0; p < md->files.size(); ++p) { + auto& file = md->files[p]; + auto const& file_path = column_chunks_file_path[p]; + for (auto& rowgroup : file.row_groups) { + for (auto& col : rowgroup.columns) { + col.file_path = file_path; + } + } + if (p == 0) { + merged_md = md->get_metadata(0); + } else { + merged_md.row_groups.insert(merged_md.row_groups.end(), + std::make_move_iterator(file.row_groups.begin()), + std::make_move_iterator(file.row_groups.end())); + merged_md.num_rows += file.num_rows; + } + } + file_ender_s fendr; + fendr.magic = parquet_magic; + fendr.footer_len = static_cast(cpw.write(merged_md)); + buffer.insert(buffer.end(), + reinterpret_cast(&fendr), + reinterpret_cast(&fendr) + sizeof(fendr)); + return std::make_unique>(std::move(buffer)); + } else { + return {nullptr}; + } return nullptr; } @@ -1672,13 +1690,14 @@ void writer::write(table_view const& table, } // Forward to implementation -std::unique_ptr> writer::close(std::string const& column_chunks_file_path) +std::unique_ptr> writer::close( + std::vector const& column_chunks_file_path) { return _impl->close(column_chunks_file_path); } std::unique_ptr> writer::merge_row_group_metadata( - const std::vector>>& metadata_list) + std::vector>> const& metadata_list) { std::vector output; CompactProtocolWriter cpw(&output); diff --git a/cpp/src/io/parquet/writer_impl.hpp b/cpp/src/io/parquet/writer_impl.hpp index add90d389e4..1d0c039f0eb 100644 --- a/cpp/src/io/parquet/writer_impl.hpp +++ b/cpp/src/io/parquet/writer_impl.hpp @@ -113,7 +113,8 @@ class writer::impl { * @return A parquet-compatible blob that contains the data for all rowgroups in the list only if * `column_chunks_file_path` is provided, else null. */ - std::unique_ptr> close(std::string const& column_chunks_file_path = ""); + std::unique_ptr> close( + std::vector const& column_chunks_file_path = {}); private: /** @@ -224,7 +225,6 @@ class writer::impl { // a single table write. this enables some internal optimizations. bool const single_write_mode = true; - std::vector buffer_; std::vector> out_sink_; }; diff --git a/cpp/tests/io/parquet_test.cpp b/cpp/tests/io/parquet_test.cpp index bcf2214a70e..635aec5a1b8 100644 --- a/cpp/tests/io/parquet_test.cpp +++ b/cpp/tests/io/parquet_test.cpp @@ -1331,7 +1331,7 @@ TEST_F(ParquetChunkedWriterTest, ManyTables) std::for_each(table_views.begin(), table_views.end(), [&writer](table_view const& tbl) { writer.write(tbl); }); - auto md = writer.close("dummy/path"); + auto md = writer.close({"dummy/path"}); CUDF_EXPECTS(md, "The returned metadata should not be null."); cudf_io::parquet_reader_options read_opts = diff --git a/python/cudf/cudf/_lib/cpp/io/parquet.pxd b/python/cudf/cudf/_lib/cpp/io/parquet.pxd index 9d95dce83bc..9525ba37897 100644 --- a/python/cudf/cudf/_lib/cpp/io/parquet.pxd +++ b/python/cudf/cudf/_lib/cpp/io/parquet.pxd @@ -73,7 +73,7 @@ cdef extern from "cudf/io/parquet.hpp" namespace "cudf::io" nogil: cudf_io_types.statistics_freq get_stats_level() except + cudf_table_view.table_view get_table() except + const cudf_io_types.table_input_metadata get_metadata() except + - string get_column_chunks_file_path() except+ + string get_column_chunks_file_paths() except+ size_t get_row_group_size_bytes() except+ size_type get_row_group_size_rows() except+ @@ -86,8 +86,8 @@ cdef extern from "cudf/io/parquet.hpp" namespace "cudf::io" nogil: void set_compression( cudf_io_types.compression_type compression ) except + - void set_column_chunks_file_path( - string column_chunks_file_path + void set_column_chunks_file_paths( + vector[string] column_chunks_file_paths ) except + void set_row_group_size_bytes(size_t val) except+ void set_row_group_size_rows(size_type val) except+ @@ -114,8 +114,8 @@ cdef extern from "cudf/io/parquet.hpp" namespace "cudf::io" nogil: parquet_writer_options_builder& compression( cudf_io_types.compression_type compression ) except + - parquet_writer_options_builder& column_chunks_file_path( - string column_chunks_file_path + parquet_writer_options_builder& column_chunks_file_paths( + vector[string] column_chunks_file_paths ) except + parquet_writer_options_builder& int96_timestamps( bool enabled @@ -190,7 +190,7 @@ cdef extern from "cudf/io/parquet.hpp" namespace "cudf::io" nogil: cudf_table_view.table_view table_, ) except+ unique_ptr[vector[uint8_t]] close( - string column_chunks_file_path, + vector[string] column_chunks_file_paths, ) except+ cdef unique_ptr[vector[uint8_t]] merge_row_group_metadata( diff --git a/python/cudf/cudf/_lib/parquet.pyx b/python/cudf/cudf/_lib/parquet.pyx index c77144ead45..03bc9b33454 100644 --- a/python/cudf/cudf/_lib/parquet.pyx +++ b/python/cudf/cudf/_lib/parquet.pyx @@ -334,10 +334,10 @@ cpdef write_parquet( cdef cudf_io_types.statistics_freq stat_freq = _get_stat_freq(statistics) cdef unique_ptr[vector[uint8_t]] out_metadata_c - cdef string c_column_chunks_file_path + cdef vector[string] c_column_chunks_file_paths cdef bool _int96_timestamps = int96_timestamps if metadata_file_path is not None: - c_column_chunks_file_path = str.encode(metadata_file_path) + c_column_chunks_file_paths.push_back(str.encode(metadata_file_path)) # Perform write cdef parquet_writer_options args = move( @@ -345,7 +345,7 @@ cpdef write_parquet( .metadata(tbl_meta.get()) .compression(comp_type) .stats_level(stat_freq) - .column_chunks_file_path(c_column_chunks_file_path) + .column_chunks_file_paths(c_column_chunks_file_paths) .int96_timestamps(_int96_timestamps) .build() ) @@ -410,18 +410,18 @@ cdef class ParquetWriter: def close(self, object metadata_file_path=None): cdef unique_ptr[vector[uint8_t]] out_metadata_c - cdef string column_chunks_file_path + cdef vector[string] column_chunks_file_paths if not self.initialized: return None # Update metadata-collection options if metadata_file_path is not None: - column_chunks_file_path = str.encode(metadata_file_path) + column_chunks_file_paths.push_back(str.encode(metadata_file_path)) with nogil: out_metadata_c = move( - self.writer.get()[0].close(column_chunks_file_path) + self.writer.get()[0].close(column_chunks_file_paths) ) if metadata_file_path is not None: From aa4582784cb1bfb9b4e488f07924032a90da3512 Mon Sep 17 00:00:00 2001 From: Devavret Makkar Date: Thu, 2 Dec 2021 00:03:38 +0530 Subject: [PATCH 12/71] Revert to using meta ctor without user_data in pyx Exception raised by writer needs to be same as that raised by pandas. If user_data is constructed earlier using pyarrow then the exception is raised early and is different --- python/cudf/cudf/_lib/parquet.pyx | 13 ++++++++----- 1 file changed, 8 insertions(+), 5 deletions(-) diff --git a/python/cudf/cudf/_lib/parquet.pyx b/python/cudf/cudf/_lib/parquet.pyx index 03bc9b33454..e824c5152de 100644 --- a/python/cudf/cudf/_lib/parquet.pyx +++ b/python/cudf/cudf/_lib/parquet.pyx @@ -301,14 +301,11 @@ cpdef write_parquet( cdef unique_ptr[cudf_io_types.data_sink] _data_sink cdef cudf_io_types.sink_info sink = make_sink_info(path, _data_sink) - pandas_metadata = generate_pandas_metadata(table, index) - user_data[str.encode("pandas")] = str.encode(pandas_metadata) - if index is True or ( index is None and not isinstance(table._index, cudf.RangeIndex) ): tv = table_view_from_table(table) - tbl_meta = make_unique[table_input_metadata](tv, move(user_data)) + tbl_meta = make_unique[table_input_metadata](tv) for level, idx_name in enumerate(table._index.names): tbl_meta.get().column_metadata[level].set_name( str.encode( @@ -318,7 +315,7 @@ cpdef write_parquet( num_index_cols_meta = len(table._index.names) else: tv = table_view_from_table(table, ignore_index=True) - tbl_meta = make_unique[table_input_metadata](tv, move(user_data)) + tbl_meta = make_unique[table_input_metadata](tv) num_index_cols_meta = 0 for i, name in enumerate(table._column_names, num_index_cols_meta): @@ -330,6 +327,12 @@ cpdef write_parquet( table[name]._column, tbl_meta.get().column_metadata[i] ) + pandas_metadata = generate_pandas_metadata(table, index) + user_data[str.encode("pandas")] = str.encode(pandas_metadata) + + # Set the table_metadata + tbl_meta.get().user_data[0] = move(user_data) + cdef cudf_io_types.compression_type comp_type = _get_comp_type(compression) cdef cudf_io_types.statistics_freq stat_freq = _get_stat_freq(statistics) From 06b264354c72b11fe37b835fe286751b707c2ac2 Mon Sep 17 00:00:00 2001 From: Devavret Makkar Date: Thu, 2 Dec 2021 01:16:05 +0530 Subject: [PATCH 13/71] Remove num_rows param and docs cleanup --- cpp/src/io/parquet/chunk_dict.cu | 24 ++-------- cpp/src/io/parquet/page_enc.cu | 77 +----------------------------- cpp/src/io/parquet/parquet_gpu.hpp | 15 +++--- cpp/src/io/parquet/writer_impl.cu | 16 ++----- cpp/src/io/parquet/writer_impl.hpp | 4 +- 5 files changed, 20 insertions(+), 116 deletions(-) diff --git a/cpp/src/io/parquet/chunk_dict.cu b/cpp/src/io/parquet/chunk_dict.cu index c00aca2ec79..f40ecbd50b8 100644 --- a/cpp/src/io/parquet/chunk_dict.cu +++ b/cpp/src/io/parquet/chunk_dict.cu @@ -95,8 +95,7 @@ struct map_find_fn { template __global__ void __launch_bounds__(block_size, 1) populate_chunk_hash_maps_kernel(cudf::detail::device_2dspan chunks, - cudf::detail::device_2dspan frags, - size_type num_rows) + cudf::detail::device_2dspan frags) { auto col_idx = blockIdx.y; auto block_x = blockIdx.x; @@ -218,8 +217,7 @@ __global__ void __launch_bounds__(block_size, 1) template __global__ void __launch_bounds__(block_size, 1) get_dictionary_indices_kernel(cudf::detail::device_2dspan chunks, - cudf::detail::device_2dspan frags, - size_type num_rows) + cudf::detail::device_2dspan frags) { auto col_idx = blockIdx.y; auto block_x = blockIdx.x; @@ -281,23 +279,13 @@ void initialize_chunk_hash_maps(device_span chunks, rmm::cuda_st void populate_chunk_hash_maps(cudf::detail::device_2dspan chunks, cudf::detail::device_2dspan frags, - size_type num_rows, rmm::cuda_stream_view stream) { constexpr int block_size = 256; - // auto const grid_x = cudf::detail::grid_1d(num_rows, max_page_fragment_size); - // auto const num_columns = chunks.size().second; - // dim3 const dim_grid(grid_x.num_blocks, num_columns); - // TODO: Is there any perf implications if the kernel is launched with x = cols, y = rows/frags? dim3 const dim_grid(frags.size().second, frags.size().first); - // Convert to a per-fragment kernel. It is like that already and I know we cannot avoid fragments - // anymore. The only other alternative is using row_bit_count to find per-row size and then use it - // to calculate rowgroup boundaries. And that one doesn't exclude null data size. Maybe in the - // future we can remove fragments and allow rowgroups to have less than 5000 rows but it's not - // important right now populate_chunk_hash_maps_kernel - <<>>(chunks, frags, num_rows); + <<>>(chunks, frags); } void collect_map_entries(device_span chunks, rmm::cuda_stream_view stream) @@ -308,17 +296,13 @@ void collect_map_entries(device_span chunks, rmm::cuda_stream_vi void get_dictionary_indices(cudf::detail::device_2dspan chunks, cudf::detail::device_2dspan frags, - size_type num_rows, rmm::cuda_stream_view stream) { constexpr int block_size = 256; - // auto const grid_x = cudf::detail::grid_1d(num_rows, max_page_fragment_size); - // auto const num_columns = chunks.size().second; - // dim3 const dim_grid(grid_x.num_blocks, num_columns); dim3 const dim_grid(frags.size().second, frags.size().first); get_dictionary_indices_kernel - <<>>(chunks, frags, num_rows); + <<>>(chunks, frags); } } // namespace gpu } // namespace parquet diff --git a/cpp/src/io/parquet/page_enc.cu b/cpp/src/io/parquet/page_enc.cu index 0b2b3a7eefd..583287c2855 100644 --- a/cpp/src/io/parquet/page_enc.cu +++ b/cpp/src/io/parquet/page_enc.cu @@ -98,17 +98,6 @@ inline __device__ uint32_t uint64_init_hash(uint64_t v) return uint32_init_hash(static_cast(v + (v >> 32))); } -/** - * @brief Initializes encoder page fragments - * - * Based on the number of rows in each fragment, populates the value count, the size of data in the - * fragment, the number of unique values, and the data size of unique values. - * - * @param[in] frag Fragment array [fragment_id][column_id] - * @param[in] col_desc Column description array [column_id] - * @param[in] num_fragments Number of fragments per column - * @param[in] num_columns Number of columns - */ // blockDim {512,1,1} template __global__ void __launch_bounds__(block_size) @@ -116,8 +105,7 @@ __global__ void __launch_bounds__(block_size) device_span col_desc, device_span partitions, device_span part_frag_offset, - uint32_t fragment_size, - uint32_t max_num_rows) // TODO: remove + uint32_t fragment_size) { __shared__ __align__(16) frag_init_state_s state_g; @@ -1902,38 +1890,20 @@ dremel_data get_dremel_data(column_view h_col, std::move(new_offsets), std::move(rep_level), std::move(def_level), leaf_data_size}; } -/** - * @brief Launches kernel for initializing encoder page fragments - * - * @param[in,out] frag Fragment array [column_id][fragment_id] - * @param[in] col_desc Column description array [column_id] - * @param[in] num_fragments Number of fragments per column - * @param[in] num_columns Number of columns - * @param[in] stream CUDA stream to use, default 0 - */ void InitPageFragments(device_2dspan frag, device_span col_desc, device_span partitions, device_span part_frag_offset, uint32_t fragment_size, - uint32_t num_rows, rmm::cuda_stream_view stream) { auto num_columns = frag.size().first; auto num_fragments_per_column = frag.size().second; dim3 dim_grid(num_columns, num_fragments_per_column); // 1 threadblock per fragment gpuInitPageFragments<512><<>>( - frag, col_desc, partitions, part_frag_offset, fragment_size, num_rows); + frag, col_desc, partitions, part_frag_offset, fragment_size); } -/** - * @brief Launches kernel for initializing fragment statistics groups - * - * @param[out] groups Statistics groups [num_columns x num_fragments] - * @param[in] fragments Page fragments [num_columns x num_fragments] - * @param[in] col_desc Column description [num_columns] - * @param[in] stream CUDA stream to use, default 0 - */ void InitFragmentStatistics(device_2dspan groups, device_2dspan fragments, device_span col_desc, @@ -1946,19 +1916,6 @@ void InitFragmentStatistics(device_2dspan groups, gpuInitFragmentStats<<>>(groups, fragments, col_desc); } -/** - * @brief Launches kernel for initializing encoder data pages - * - * @param[in,out] chunks Column chunks [rowgroup][column] - * @param[out] pages Encode page array (null if just counting pages) - * @param[in] col_desc Column description array [column_id] - * @param[in] num_rowgroups Number of fragments per column - * @param[in] num_columns Number of columns - * @param[out] page_grstats Setup for page-level stats - * @param[out] chunk_grstats Setup for chunk-level stats - * @param[in] max_page_comp_data_size Calculated maximum compressed data size of pages - * @param[in] stream CUDA stream to use, default 0 - */ void InitEncoderPages(device_2dspan chunks, device_span pages, device_span col_desc, @@ -1974,14 +1931,6 @@ void InitEncoderPages(device_2dspan chunks, chunks, pages, col_desc, page_grstats, chunk_grstats, max_page_comp_data_size, num_columns); } -/** - * @brief Launches kernel for packing column data into parquet pages - * - * @param[in,out] pages Device array of EncPages (unordered) - * @param[out] comp_in Optionally initializes compressor input params - * @param[out] comp_stat Optionally initializes compressor status - * @param[in] stream CUDA stream to use, default 0 - */ void EncodePages(device_span pages, device_span comp_in, device_span comp_stat, @@ -1993,26 +1942,11 @@ void EncodePages(device_span pages, gpuEncodePages<128><<>>(pages, comp_in, comp_stat); } -/** - * @brief Launches kernel to make the compressed vs uncompressed chunk-level decision - * - * @param[in,out] chunks Column chunks - * @param[in] stream CUDA stream to use, default 0 - */ void DecideCompression(device_span chunks, rmm::cuda_stream_view stream) { gpuDecideCompression<<>>(chunks); } -/** - * @brief Launches kernel to encode page headers - * - * @param[in,out] pages Device array of EncPages - * @param[in] comp_stat Compressor status or nullptr if no compression - * @param[in] page_stats Optional page-level statistics to be included in page header - * @param[in] chunk_stats Optional chunk-level statistics to be encoded - * @param[in] stream CUDA stream to use, default 0 - */ void EncodePageHeaders(device_span pages, device_span comp_stat, device_span page_stats, @@ -2025,13 +1959,6 @@ void EncodePageHeaders(device_span pages, pages, comp_stat, page_stats, chunk_stats); } -/** - * @brief Launches kernel to gather pages to a single contiguous block per chunk - * - * @param[in,out] chunks Column chunks - * @param[in] pages Device array of EncPages - * @param[in] stream CUDA stream to use, default 0 - */ void GatherPages(device_span chunks, device_span pages, rmm::cuda_stream_view stream) diff --git a/cpp/src/io/parquet/parquet_gpu.hpp b/cpp/src/io/parquet/parquet_gpu.hpp index e5498018a4c..e62983e7e39 100644 --- a/cpp/src/io/parquet/parquet_gpu.hpp +++ b/cpp/src/io/parquet/parquet_gpu.hpp @@ -491,12 +491,14 @@ dremel_data get_dremel_data(column_view h_col, /** * @brief Launches kernel for initializing encoder page fragments * + * Based on the number of rows in each fragment, populates the value count, the size of data in the + * fragment, the number of unique values, and the data size of unique values. + * * @param[out] frag Fragment array [column_id][fragment_id] * @param[in] col_desc Column description array [column_id] - * @param[in] num_fragments Number of fragments per column - * @param[in] num_columns Number of columns + * @param[in] partitions Information about partitioning of table + * @param[in] first_frag_in_part A Partition's offset into fragment array * @param[in] fragment_size Number of rows per fragment - * @param[in] num_rows Number of rows per column * @param[in] stream CUDA stream to use */ void InitPageFragments(cudf::detail::device_2dspan frag, @@ -504,7 +506,6 @@ void InitPageFragments(cudf::detail::device_2dspan frag, device_span partitions, device_span first_frag_in_part, uint32_t fragment_size, - uint32_t num_rows, rmm::cuda_stream_view stream); /** @@ -532,12 +533,11 @@ void initialize_chunk_hash_maps(device_span chunks, rmm::cuda_st * @brief Insert chunk values into their respective hash maps * * @param chunks Column chunks [rowgroup][column] - * @param num_rows Number of rows per column + * @param frags Column fragments * @param stream CUDA stream to use */ void populate_chunk_hash_maps(cudf::detail::device_2dspan chunks, cudf::detail::device_2dspan frags, - size_type num_rows, rmm::cuda_stream_view stream); /** @@ -558,12 +558,11 @@ void collect_map_entries(device_span chunks, rmm::cuda_stream_vi * col[row] == col[dict_data[dict_index[row - chunk.start_row]]] * * @param chunks Column chunks [rowgroup][column] - * @param num_rows Number of rows per column + * @param frags Column fragments * @param stream CUDA stream to use */ void get_dictionary_indices(cudf::detail::device_2dspan chunks, cudf::detail::device_2dspan frags, - size_type num_rows, rmm::cuda_stream_view stream); /** diff --git a/cpp/src/io/parquet/writer_impl.cu b/cpp/src/io/parquet/writer_impl.cu index 29aebed598b..1439946b4b1 100644 --- a/cpp/src/io/parquet/writer_impl.cu +++ b/cpp/src/io/parquet/writer_impl.cu @@ -772,7 +772,6 @@ void writer::impl::init_page_fragments(cudf::detail::hostdevice_2dvector col_desc, std::vector> partitions, device_span part_frag_offset, - uint32_t num_rows, uint32_t fragment_size) { // TODO: partitions as pairs aren't convertible to device_uvector because device_uvector does not @@ -786,8 +785,7 @@ void writer::impl::init_page_fragments(cudf::detail::hostdevice_2dvector& chu auto build_chunk_dictionaries(hostdevice_2dvector& chunks, host_span col_desc, device_2dspan frags, - uint32_t num_rows, rmm::cuda_stream_view stream) { // At this point, we know all chunks and their sizes. We want to allocate dictionaries for each @@ -850,7 +847,7 @@ auto build_chunk_dictionaries(hostdevice_2dvector& chunks, chunks.host_to_device(stream); gpu::initialize_chunk_hash_maps(chunks.device_view().flat_view(), stream); - gpu::populate_chunk_hash_maps(chunks, frags, num_rows, stream); + gpu::populate_chunk_hash_maps(chunks, frags, stream); chunks.device_to_host(stream, true); @@ -899,7 +896,7 @@ auto build_chunk_dictionaries(hostdevice_2dvector& chunks, } chunks.host_to_device(stream); gpu::collect_map_entries(chunks.device_view().flat_view(), stream); - gpu::get_dictionary_indices(chunks.device_view(), frags, num_rows, stream); + gpu::get_dictionary_indices(chunks.device_view(), frags, stream); return std::make_pair(std::move(dict_data), std::move(dict_index)); } @@ -1128,9 +1125,6 @@ void writer::impl::write(table_view const& table, CUDF_EXPECTS(not closed, "Data has already been flushed to out and closed"); if (partitions.empty()) { partitions.push_back({0, table.num_rows()}); } - // TODO: remove - size_type num_rows = table.num_rows(); - if (not table_meta) { table_meta = std::make_unique(table); } // Fill unnamed columns' names in table_meta @@ -1243,7 +1237,7 @@ void writer::impl::write(table_view const& table, col_desc, *parent_column_table_device_view, stream); init_page_fragments( - fragments, col_desc, partitions, d_part_frag_offset, num_rows, max_page_fragment_size); + fragments, col_desc, partitions, d_part_frag_offset, max_page_fragment_size); } // TODO: size_type @@ -1370,7 +1364,7 @@ void writer::impl::write(table_view const& table, // Pass fragments hd_vec to build_chunk_dictionaries fragments.host_to_device(stream); - auto dict_info_owner = build_chunk_dictionaries(chunks, col_desc, fragments, num_rows, stream); + auto dict_info_owner = build_chunk_dictionaries(chunks, col_desc, fragments, stream); for (size_t p = 0; p < partitions.size(); p++) { for (int rg = 0; rg < num_rg_in_part[p]; rg++) { size_t global_rg = global_rowgroup_base[p] + rg; diff --git a/cpp/src/io/parquet/writer_impl.hpp b/cpp/src/io/parquet/writer_impl.hpp index 1d0c039f0eb..2311dc5379d 100644 --- a/cpp/src/io/parquet/writer_impl.hpp +++ b/cpp/src/io/parquet/writer_impl.hpp @@ -122,14 +122,14 @@ class writer::impl { * * @param frag Destination page fragments * @param col_desc column description array - * @param num_rows Total number of rows + * @param[in] partitions Information about partitioning of table + * @param[in] part_frag_offset A Partition's offset into fragment array * @param fragment_size Number of rows per fragment */ void init_page_fragments(hostdevice_2dvector& frag, device_span col_desc, std::vector> partitions, device_span part_frag_offset, - uint32_t num_rows, uint32_t fragment_size); /** From fffb41efeb6feb50565fc83475445fc2cacb0c15 Mon Sep 17 00:00:00 2001 From: Devavret Makkar Date: Thu, 2 Dec 2021 01:32:30 +0530 Subject: [PATCH 14/71] orc use table meta ctor with single user_data --- cpp/src/io/parquet/page_enc.cu | 2 -- python/cudf/cudf/_lib/orc.pyx | 19 ++++++++++++------- 2 files changed, 12 insertions(+), 9 deletions(-) diff --git a/cpp/src/io/parquet/page_enc.cu b/cpp/src/io/parquet/page_enc.cu index 583287c2855..9601b1272af 100644 --- a/cpp/src/io/parquet/page_enc.cu +++ b/cpp/src/io/parquet/page_enc.cu @@ -49,8 +49,6 @@ constexpr uint32_t rle_buffer_size = (1 << 9); struct frag_init_state_s { parquet_column_device_view col; PageFragment frag; - // TODO: replace this with frag.start_value - size_type start_value_idx; }; struct page_enc_state_s { diff --git a/python/cudf/cudf/_lib/orc.pyx b/python/cudf/cudf/_lib/orc.pyx index 3ce9d4f0bcc..99c50c51ee8 100644 --- a/python/cudf/cudf/_lib/orc.pyx +++ b/python/cudf/cudf/_lib/orc.pyx @@ -3,6 +3,7 @@ import cudf from libcpp cimport bool, int +from libcpp.map cimport map from libcpp.memory cimport make_unique, unique_ptr from libcpp.string cimport string from libcpp.utility cimport move @@ -311,14 +312,19 @@ cdef class ORCWriter: cdef table_view tv # Set the table_metadata + cdef map[string, string] user_data + pandas_metadata = generate_pandas_metadata(table, self.index) + user_data[str.encode("pandas")] = str.encode(pandas_metadata) + num_index_cols_meta = 0 self.tbl_meta = make_unique[table_input_metadata]( - table_view_from_table(table, ignore_index=True) + table_view_from_table(table, ignore_index=True), + user_data, ) if self.index is not False: if isinstance(table._index, cudf.core.multiindex.MultiIndex): tv = table_view_from_table(table) - self.tbl_meta = make_unique[table_input_metadata](tv) + self.tbl_meta = make_unique[table_input_metadata](tv, user_data) for level, idx_name in enumerate(table._index.names): self.tbl_meta.get().column_metadata[level].set_name( (str.encode(idx_name)) @@ -327,7 +333,10 @@ cdef class ORCWriter: else: if table._index.name is not None: tv = table_view_from_table(table) - self.tbl_meta = make_unique[table_input_metadata](tv) + self.tbl_meta = make_unique[table_input_metadata]( + tv, + user_data, + ) self.tbl_meta.get().column_metadata[0].set_name( str.encode(table._index.name) ) @@ -339,10 +348,6 @@ cdef class ORCWriter: table[name]._column, self.tbl_meta.get().column_metadata[i] ) - pandas_metadata = generate_pandas_metadata(table, self.index) - self.tbl_meta.get().user_data[0][str.encode("pandas")] = \ - str.encode(pandas_metadata) - cdef chunked_orc_writer_options args with nogil: args = move( From ecd3aa5cc9a94ad7cc9d256f0bc598506e0a9d53 Mon Sep 17 00:00:00 2001 From: Devavret Makkar Date: Thu, 2 Dec 2021 05:15:21 +0530 Subject: [PATCH 15/71] Small size_type cleanups --- cpp/src/io/parquet/parquet_gpu.hpp | 2 +- cpp/src/io/parquet/writer_impl.cu | 36 +++++++++--------------------- 2 files changed, 12 insertions(+), 26 deletions(-) diff --git a/cpp/src/io/parquet/parquet_gpu.hpp b/cpp/src/io/parquet/parquet_gpu.hpp index e62983e7e39..3207c624c31 100644 --- a/cpp/src/io/parquet/parquet_gpu.hpp +++ b/cpp/src/io/parquet/parquet_gpu.hpp @@ -341,7 +341,7 @@ struct EncColumnChunk { uint32_t compressed_size; //!< Compressed buffer size uint32_t max_page_data_size; //!< Max data size (excluding header) of any page in this chunk uint32_t page_headers_size; //!< Sum of size of all page headers - uint32_t start_row; //!< First row of chunk + size_type start_row; //!< First row of chunk uint32_t num_rows; //!< Number of rows in chunk size_type num_values; //!< Number of values in chunk. Different from num_rows for nested types uint32_t first_fragment; //!< First fragment of chunk diff --git a/cpp/src/io/parquet/writer_impl.cu b/cpp/src/io/parquet/writer_impl.cu index 1439946b4b1..164f6acf134 100644 --- a/cpp/src/io/parquet/writer_impl.cu +++ b/cpp/src/io/parquet/writer_impl.cu @@ -1240,7 +1240,6 @@ void writer::impl::write(table_view const& table, fragments, col_desc, partitions, d_part_frag_offset, max_page_fragment_size); } - // TODO: size_type std::vector global_rowgroup_base; std::transform(md->files.begin(), md->files.end(), @@ -1248,28 +1247,25 @@ void writer::impl::write(table_view const& table, [](auto const& part) { return part.row_groups.size(); }); // Decide row group boundaries based on uncompressed data size - // size_t rowgroup_size = 0; size_type num_rowgroups = 0; - std::vector num_frag_in_rg; // TODO: Why do we need this? std::vector num_rg_in_part(partitions.size()); for (size_t p = 0; p < partitions.size(); ++p) { - // TODO: size_type - int curr_rg_num_rows = 0; - int curr_rg_data_size = 0; - int first_frag_in_rg = part_frag_offset[p]; - int last_frag_in_part = part_frag_offset[p + 1] - 1; - for (int f = first_frag_in_rg; f <= last_frag_in_part; ++f) { - int fragment_data_size = 0; + size_type curr_rg_num_rows = 0; + size_t curr_rg_data_size = 0; + int first_frag_in_rg = part_frag_offset[p]; + int last_frag_in_part = part_frag_offset[p + 1] - 1; + for (auto f = first_frag_in_rg; f <= last_frag_in_part; ++f) { + size_t fragment_data_size = 0; for (auto c = 0; c < num_columns; c++) { fragment_data_size += fragments[c][f].fragment_data_size; } - int fragment_num_rows = fragments[0][f].num_rows; + size_type fragment_num_rows = fragments[0][f].num_rows; // If the fragment size gets larger than rg limit then break off a rg if (f > first_frag_in_rg && // There has to be at least one fragment in row group - (curr_rg_data_size + fragment_data_size > (int)max_row_group_size || - curr_rg_num_rows + fragment_num_rows > (int)max_row_group_rows)) { + (curr_rg_data_size + fragment_data_size > max_row_group_size || + curr_rg_num_rows + fragment_num_rows > max_row_group_rows)) { auto& rg = md->files[p].row_groups.emplace_back(); rg.num_rows = curr_rg_num_rows; num_rowgroups++; @@ -1311,11 +1307,9 @@ void writer::impl::write(table_view const& table, auto const num_chunks = num_rowgroups * num_columns; hostdevice_2dvector chunks(num_rowgroups, num_columns, stream); - // TODO: alternative method is to make this a loop ovr only rg and get p using rg_to_part for (size_t p = 0; p < partitions.size(); ++p) { - // TODO: size_type - size_t f = part_frag_offset[p]; - size_t start_row = partitions[p].first; + int f = part_frag_offset[p]; + size_type start_row = partitions[p].first; for (int r = 0; r < num_rg_in_part[p]; r++) { size_t global_r = global_rowgroup_base[p] + r; // Number of rowgroups already in file/part uint32_t fragments_in_chunk = util::div_rounding_up_unsafe( @@ -1403,14 +1397,6 @@ void writer::impl::write(table_view const& table, num_rg_in_part.begin(), num_rg_in_part.end(), std::back_inserter(part_end_rg)); std::vector rg_to_part; auto it = thrust::make_counting_iterator(0); - // std::vector range(num_rowgroups); - // std::iota(range.begin(), range.end(), 0); - // thrust::upper_bound(thrust::host, - // part_end_rg.begin(), - // part_end_rg.end(), - // range.begin(), - // range.end(), - // rg_to_part.begin()); std::transform(it, it + num_rowgroups, std::back_inserter(rg_to_part), [&](auto i) { return std::upper_bound(part_end_rg.begin(), part_end_rg.end(), i) - part_end_rg.begin(); }); From 950f505b8eb56962af974ad58fe119fbf2cbe37c Mon Sep 17 00:00:00 2001 From: Devavret Makkar Date: Thu, 2 Dec 2021 17:20:16 +0530 Subject: [PATCH 16/71] Misc cleanups --- cpp/src/io/parquet/page_enc.cu | 3 +-- cpp/src/io/parquet/parquet_gpu.hpp | 2 -- cpp/src/io/parquet/writer_impl.cu | 7 +------ python/cudf/cudf/_lib/orc.pyx | 4 +++- 4 files changed, 5 insertions(+), 11 deletions(-) diff --git a/cpp/src/io/parquet/page_enc.cu b/cpp/src/io/parquet/page_enc.cu index 9601b1272af..a044127c473 100644 --- a/cpp/src/io/parquet/page_enc.cu +++ b/cpp/src/io/parquet/page_enc.cu @@ -132,10 +132,9 @@ __global__ void __launch_bounds__(block_size) s->frag.fragment_data_size = 0; s->frag.dict_data_size = 0; - size_type end_row = s->frag.start_row + s->frag.num_rows; auto col = *(s->col.parent_column); s->frag.start_value_idx = row_to_value_idx(s->frag.start_row, col); - size_type end_value_idx = row_to_value_idx(end_row, col); + size_type end_value_idx = row_to_value_idx(s->frag.start_row + s->frag.num_rows, col); s->frag.num_leaf_values = end_value_idx - s->frag.start_value_idx; if (s->col.level_offsets != nullptr) { diff --git a/cpp/src/io/parquet/parquet_gpu.hpp b/cpp/src/io/parquet/parquet_gpu.hpp index 3207c624c31..dc826168d92 100644 --- a/cpp/src/io/parquet/parquet_gpu.hpp +++ b/cpp/src/io/parquet/parquet_gpu.hpp @@ -331,8 +331,6 @@ struct EncPage; struct EncColumnChunk { parquet_column_device_view const* col_desc; //!< Column description size_type col_desc_id; - - // Add a num fragments PageFragment* fragments; //!< First fragment in chunk uint8_t* uncompressed_bfr; //!< Uncompressed page data uint8_t* compressed_bfr; //!< Compressed page data diff --git a/cpp/src/io/parquet/writer_impl.cu b/cpp/src/io/parquet/writer_impl.cu index 164f6acf134..97d167a62a1 100644 --- a/cpp/src/io/parquet/writer_impl.cu +++ b/cpp/src/io/parquet/writer_impl.cu @@ -1168,9 +1168,6 @@ void writer::impl::write(table_view const& table, md->column_order_listsize = (stats_granularity_ != statistics_freq::STATISTICS_NONE) ? num_columns : 0; - // Think about how this will be passed. Currently it is passed in table_input_metadata which is - // only passed once as part of args to writer ctor. Now this would need to be passed per sink. - // But we only need them once. Ask in review for (size_t p = 0; p < table_meta->user_data.size(); ++p) { std::transform(table_meta->user_data[p].begin(), table_meta->user_data[p].end(), @@ -1220,7 +1217,6 @@ void writer::impl::write(table_view const& table, size_type num_fragments = std::reduce(num_frag_in_part.begin(), num_frag_in_part.end()); - // TODO: better comments, size_type std::vector part_frag_offset; // Store the idx of the first fragment in each partition std::exclusive_scan( num_frag_in_part.begin(), num_frag_in_part.end(), std::back_inserter(part_frag_offset), 0); @@ -1247,7 +1243,7 @@ void writer::impl::write(table_view const& table, [](auto const& part) { return part.row_groups.size(); }); // Decide row group boundaries based on uncompressed data size - size_type num_rowgroups = 0; + int num_rowgroups = 0; std::vector num_rg_in_part(partitions.size()); for (size_t p = 0; p < partitions.size(); ++p) { @@ -1356,7 +1352,6 @@ void writer::impl::write(table_view const& table, } } - // Pass fragments hd_vec to build_chunk_dictionaries fragments.host_to_device(stream); auto dict_info_owner = build_chunk_dictionaries(chunks, col_desc, fragments, stream); for (size_t p = 0; p < partitions.size(); p++) { diff --git a/python/cudf/cudf/_lib/orc.pyx b/python/cudf/cudf/_lib/orc.pyx index 99c50c51ee8..22547909528 100644 --- a/python/cudf/cudf/_lib/orc.pyx +++ b/python/cudf/cudf/_lib/orc.pyx @@ -324,7 +324,9 @@ cdef class ORCWriter: if self.index is not False: if isinstance(table._index, cudf.core.multiindex.MultiIndex): tv = table_view_from_table(table) - self.tbl_meta = make_unique[table_input_metadata](tv, user_data) + self.tbl_meta = make_unique[table_input_metadata]( + tv, user_data, + ) for level, idx_name in enumerate(table._index.names): self.tbl_meta.get().column_metadata[level].set_name( (str.encode(idx_name)) From 1d55d1a9f326d570b4dff9661ad942c5557057d4 Mon Sep 17 00:00:00 2001 From: Devavret Makkar Date: Wed, 8 Dec 2021 05:12:11 +0530 Subject: [PATCH 17/71] API changes --- cpp/include/cudf/io/detail/parquet.hpp | 3 +-- cpp/include/cudf/io/parquet.hpp | 29 +++++++++++++------------- cpp/include/cudf/io/types.hpp | 5 +++++ cpp/src/io/functions.cpp | 4 ++-- cpp/src/io/parquet/page_enc.cu | 4 ++-- cpp/src/io/parquet/parquet_gpu.hpp | 7 +------ cpp/src/io/parquet/writer_impl.cu | 28 +++++++------------------ cpp/src/io/parquet/writer_impl.hpp | 4 ++-- cpp/tests/io/parquet_test.cpp | 10 +++++---- 9 files changed, 42 insertions(+), 52 deletions(-) diff --git a/cpp/include/cudf/io/detail/parquet.hpp b/cpp/include/cudf/io/detail/parquet.hpp index 3c0f4ada247..561ce2d5b60 100644 --- a/cpp/include/cudf/io/detail/parquet.hpp +++ b/cpp/include/cudf/io/detail/parquet.hpp @@ -132,8 +132,7 @@ class writer { * @param[in] partitions Optional partitions to divide the table into. If specified, must be same * size as number of sinks. */ - void write(table_view const& table, - std::vector> const& partitions = {}); + void write(table_view const& table, std::vector const& partitions = {}); /** * @brief Finishes the chunked/streamed write process. diff --git a/cpp/include/cudf/io/parquet.hpp b/cpp/include/cudf/io/parquet.hpp index 7e4406648df..61a1f76d986 100644 --- a/cpp/include/cudf/io/parquet.hpp +++ b/cpp/include/cudf/io/parquet.hpp @@ -395,7 +395,7 @@ class parquet_writer_options { // Sets of columns to output table_view _table; // Partitions described as {start_row, num_rows} pairs - std::vector> _partitions; + std::vector _partitions; // Optional associated metadata table_input_metadata const* _metadata = nullptr; // Parquet writer can write INT96 or TIMESTAMP_MICROS. Defaults to TIMESTAMP_MICROS. @@ -469,7 +469,7 @@ class parquet_writer_options { /** * @brief Returns partitions. */ - std::vector> get_partitions() const { return _partitions; } + std::vector const& get_partitions() const { return _partitions; } /** * @brief Returns associated metadata. @@ -484,7 +484,7 @@ class parquet_writer_options { /** * @brief Returns Column chunks file paths to be set in the raw output metadata. */ - std::vector get_column_chunks_file_paths() const + std::vector const& get_column_chunks_file_paths() const { return _column_chunks_file_paths; } @@ -505,9 +505,9 @@ class parquet_writer_options { * @param partitions Partitions of input table in {start_row, num_rows} pairs. If specified, must * be same size as number of sinks in sink_info */ - void set_partitions(std::vector> const& partitions) + void set_partitions(std::vector const& partitions) { - _partitions = partitions; + _partitions = std::move(partitions); } /** @@ -542,11 +542,12 @@ class parquet_writer_options { /** * @brief Sets column chunks file path to be set in the raw output metadata. * - * @param file_paths Vector of Strings which indicates file path. + * @param file_paths Vector of Strings which indicates file path. Must be same size as partitions + * if partitions are specified */ - void set_column_chunks_file_paths(std::vector const& file_paths) + void set_column_chunks_file_paths(std::vector file_paths) { - _column_chunks_file_paths = file_paths; + _column_chunks_file_paths = std::move(file_paths); } /** @@ -601,10 +602,9 @@ class parquet_writer_options_builder { * be same size as number of sinks in sink_info * @return this for chaining. */ - parquet_writer_options_builder& partitions( - std::vector> partitions) + parquet_writer_options_builder& partitions(std::vector partitions) { - options._partitions = partitions; + options._partitions = std::move(partitions); return *this; } @@ -647,7 +647,8 @@ class parquet_writer_options_builder { /** * @brief Sets column chunks file path to be set in the raw output metadata. * - * @param file_paths Vector of Strings which indicates file path. + * @param file_paths Vector of Strings which indicates file path. Must be same size as partitions + * if partitions are specified * @return this for chaining. */ parquet_writer_options_builder& column_chunks_file_paths( @@ -1028,8 +1029,8 @@ class parquet_chunked_writer { * size as number of sinks. * @return returns reference of the class object */ - parquet_chunked_writer& write( - table_view const& table, std::vector> const& partitions = {}); + parquet_chunked_writer& write(table_view const& table, + std::vector const& partitions = {}); /** * @brief Finishes the chunked/streamed write process. diff --git a/cpp/include/cudf/io/types.hpp b/cpp/include/cudf/io/types.hpp index 87c5e999776..23533b82d0d 100644 --- a/cpp/include/cudf/io/types.hpp +++ b/cpp/include/cudf/io/types.hpp @@ -414,5 +414,10 @@ class table_input_metadata { user_data; //!< Format-dependent metadata as key-values pairs. One per output file }; +struct partition_info { + size_type start_row; + size_type num_rows; +}; + } // namespace io } // namespace cudf diff --git a/cpp/src/io/functions.cpp b/cpp/src/io/functions.cpp index f6ead9084ca..77bf733a501 100644 --- a/cpp/src/io/functions.cpp +++ b/cpp/src/io/functions.cpp @@ -492,8 +492,8 @@ parquet_chunked_writer::parquet_chunked_writer(chunked_parquet_writer_options co /** * @copydoc cudf::io::parquet_chunked_writer::write */ -parquet_chunked_writer& parquet_chunked_writer::write( - table_view const& table, std::vector> const& partitions) +parquet_chunked_writer& parquet_chunked_writer::write(table_view const& table, + std::vector const& partitions) { CUDF_FUNC_RANGE(); diff --git a/cpp/src/io/parquet/page_enc.cu b/cpp/src/io/parquet/page_enc.cu index 678b1ef888e..79bcdf7d57f 100644 --- a/cpp/src/io/parquet/page_enc.cu +++ b/cpp/src/io/parquet/page_enc.cu @@ -102,7 +102,7 @@ template __global__ void __launch_bounds__(block_size) gpuInitPageFragments(device_2dspan frag, device_span col_desc, - device_span partitions, + device_span partitions, device_span part_frag_offset, uint32_t fragment_size) { @@ -1881,7 +1881,7 @@ dremel_data get_dremel_data(column_view h_col, void InitPageFragments(device_2dspan frag, device_span col_desc, - device_span partitions, + device_span partitions, device_span part_frag_offset, uint32_t fragment_size, rmm::cuda_stream_view stream) diff --git a/cpp/src/io/parquet/parquet_gpu.hpp b/cpp/src/io/parquet/parquet_gpu.hpp index dc826168d92..4c82db36fe1 100644 --- a/cpp/src/io/parquet/parquet_gpu.hpp +++ b/cpp/src/io/parquet/parquet_gpu.hpp @@ -250,11 +250,6 @@ struct parquet_column_device_view : stats_column_desc { //!< col.nullable() in case of chunked writing. }; -struct partition_info { - size_type start_row; - size_type num_rows; -}; - constexpr int max_page_fragment_size = 5000; //!< Max number of rows in a page fragment struct EncColumnChunk; @@ -501,7 +496,7 @@ dremel_data get_dremel_data(column_view h_col, */ void InitPageFragments(cudf::detail::device_2dspan frag, device_span col_desc, - device_span partitions, + device_span partitions, device_span first_frag_in_part, uint32_t fragment_size, rmm::cuda_stream_view stream); diff --git a/cpp/src/io/parquet/writer_impl.cu b/cpp/src/io/parquet/writer_impl.cu index e9cdbcf9219..b4949aad04a 100644 --- a/cpp/src/io/parquet/writer_impl.cu +++ b/cpp/src/io/parquet/writer_impl.cu @@ -765,21 +765,11 @@ gpu::parquet_column_device_view parquet_column_view::get_device_view( void writer::impl::init_page_fragments(cudf::detail::hostdevice_2dvector& frag, device_span col_desc, - std::vector> partitions, + std::vector partitions, device_span part_frag_offset, uint32_t fragment_size) { - // TODO: partitions as pairs aren't convertible to device_uvector because device_uvector does not - // allow non-trivially-copyable types and std::pair has a copy constructor. Find a way to expose - // gpu::partition_info in public header and accept the partitions as that. OR at least make this - // conversion early on in the write() call so that we don't have to do part.first and part.second - // everywhere - std::vector h_partitions; - std::transform( - partitions.begin(), partitions.end(), std::back_inserter(h_partitions), [](auto const& part) { - return gpu::partition_info{part.first, part.second}; - }); - auto d_partitions = cudf::detail::make_device_uvector_async(h_partitions, stream); + auto d_partitions = cudf::detail::make_device_uvector_async(partitions, stream); gpu::InitPageFragments(frag, col_desc, d_partitions, part_frag_offset, fragment_size, stream); frag.device_to_host(stream, true); } @@ -1113,8 +1103,7 @@ void writer::impl::init_state() } } -void writer::impl::write(table_view const& table, - std::vector> partitions) +void writer::impl::write(table_view const& table, std::vector partitions) { last_write_successful = false; CUDF_EXPECTS(not closed, "Data has already been flushed to out and closed"); @@ -1158,7 +1147,7 @@ void writer::impl::write(table_view const& table, md = std::make_unique(partitions.size()); md->version = 1; for (size_t i = 0; i < partitions.size(); ++i) { - md->files[i].num_rows = partitions[i].second; + md->files[i].num_rows = partitions[i].num_rows; } md->column_order_listsize = (stats_granularity_ != statistics_freq::STATISTICS_NONE) ? num_columns : 0; @@ -1179,7 +1168,7 @@ void writer::impl::write(table_view const& table, // increment num rows for (size_t i = 0; i < partitions.size(); ++i) { - md->files[i].num_rows += partitions[i].second; + md->files[i].num_rows += partitions[i].num_rows; } } // Create table_device_view so that corresponding column_device_view data @@ -1207,7 +1196,7 @@ void writer::impl::write(table_view const& table, partitions.end(), std::back_inserter(num_frag_in_part), [](auto const& part) { - return util::div_rounding_up_unsafe(part.second, max_page_fragment_size); + return util::div_rounding_up_unsafe(part.num_rows, max_page_fragment_size); }); size_type num_fragments = std::reduce(num_frag_in_part.begin(), num_frag_in_part.end()); @@ -1300,7 +1289,7 @@ void writer::impl::write(table_view const& table, for (size_t p = 0; p < partitions.size(); ++p) { int f = part_frag_offset[p]; - size_type start_row = partitions[p].first; + size_type start_row = partitions[p].start_row; for (int r = 0; r < num_rg_in_part[p]; r++) { size_t global_r = global_rowgroup_base[p] + r; // Number of rowgroups already in file/part uint32_t fragments_in_chunk = util::div_rounding_up_unsafe( @@ -1653,8 +1642,7 @@ writer::writer(std::vector> sinks, writer::~writer() = default; // Forward to implementation -void writer::write(table_view const& table, - std::vector> const& partitions) +void writer::write(table_view const& table, std::vector const& partitions) { _impl->write(table, partitions); } diff --git a/cpp/src/io/parquet/writer_impl.hpp b/cpp/src/io/parquet/writer_impl.hpp index 2311dc5379d..36e587a9b6d 100644 --- a/cpp/src/io/parquet/writer_impl.hpp +++ b/cpp/src/io/parquet/writer_impl.hpp @@ -104,7 +104,7 @@ class writer::impl { * * @param[in] table The table information to be written */ - void write(table_view const& table, std::vector> partitions); + void write(table_view const& table, std::vector partitions); /** * @brief Finishes the chunked/streamed write process. @@ -128,7 +128,7 @@ class writer::impl { */ void init_page_fragments(hostdevice_2dvector& frag, device_span col_desc, - std::vector> partitions, + std::vector partitions, device_span part_frag_offset, uint32_t fragment_size); diff --git a/cpp/tests/io/parquet_test.cpp b/cpp/tests/io/parquet_test.cpp index 635aec5a1b8..9ce79c7437c 100644 --- a/cpp/tests/io/parquet_test.cpp +++ b/cpp/tests/io/parquet_test.cpp @@ -1178,11 +1178,13 @@ TEST_F(ParquetWriterTest, PartitionedWrite) auto filepath1 = temp_env->get_temp_filepath("PartitionedWrite1.parquet"); auto filepath2 = temp_env->get_temp_filepath("PartitionedWrite2.parquet"); - auto partition1 = std::make_pair(10, 1024 * 1024); - auto partition2 = std::make_pair(20 * 1024 + 7, 3 * 1024 * 1024); + auto partition1 = cudf::io::partition_info{10, 1024 * 1024}; + auto partition2 = cudf::io::partition_info{20 * 1024 + 7, 3 * 1024 * 1024}; - auto expected1 = cudf::slice(*source, {partition1.first, partition1.first + partition1.second}); - auto expected2 = cudf::slice(*source, {partition2.first, partition2.first + partition2.second}); + auto expected1 = + cudf::slice(*source, {partition1.start_row, partition1.start_row + partition1.num_rows}); + auto expected2 = + cudf::slice(*source, {partition2.start_row, partition2.start_row + partition2.num_rows}); cudf_io::parquet_writer_options args = cudf_io::parquet_writer_options::builder( From 387c2ac2abff31413f193a074613316be8c943a1 Mon Sep 17 00:00:00 2001 From: Devavret Makkar Date: Wed, 8 Dec 2021 19:30:47 +0530 Subject: [PATCH 18/71] Take user_data out of table_input_metadata --- cpp/include/cudf/io/orc.hpp | 59 ++++++++++++++++++++++++++ cpp/include/cudf/io/parquet.hpp | 66 ++++++++++++++++++++++++++++++ cpp/include/cudf/io/types.hpp | 23 ----------- cpp/src/io/functions.cpp | 14 ------- cpp/src/io/orc/writer_impl.cu | 15 +++---- cpp/src/io/orc/writer_impl.hpp | 2 + cpp/src/io/parquet/writer_impl.cu | 8 ++-- cpp/src/io/parquet/writer_impl.hpp | 2 + cpp/tests/io/parquet_test.cpp | 6 +-- 9 files changed, 143 insertions(+), 52 deletions(-) diff --git a/cpp/include/cudf/io/orc.hpp b/cpp/include/cudf/io/orc.hpp index 3bc2e6c9ef2..05158e3d55a 100644 --- a/cpp/include/cudf/io/orc.hpp +++ b/cpp/include/cudf/io/orc.hpp @@ -430,6 +430,8 @@ class orc_writer_options { table_view _table; // Optional associated metadata const table_input_metadata* _metadata = nullptr; + // Optional footer key_value_metadata + std::map _user_data; friend orc_writer_options_builder; @@ -506,6 +508,11 @@ class orc_writer_options { */ table_input_metadata const* get_metadata() const { return _metadata; } + /** + * @brief Returns Key-Value footer metadata information. + */ + std::map const& get_key_value_metadata() const { return _user_data; } + // Setters /** @@ -567,6 +574,16 @@ class orc_writer_options { * @param meta Associated metadata. */ void set_metadata(table_input_metadata const* meta) { _metadata = meta; } + + /** + * @brief Sets metadata. + * + * @param metadata Key-Value footer metadata + */ + void set_key_value_metadata(std::map metadata) + { + _user_data = std::move(metadata); + } }; class orc_writer_options_builder { @@ -674,6 +691,18 @@ class orc_writer_options_builder { return *this; } + /** + * @brief Sets Key-Value footer metadata. + * + * @param metadata Key-Value footer metadata + * @return this for chaining. + */ + orc_writer_options_builder& key_value_metadata(std::map metadata) + { + options._user_data = std::move(metadata); + return *this; + } + /** * @brief move orc_writer_options member once it's built. */ @@ -729,6 +758,8 @@ class chunked_orc_writer_options { size_type _row_index_stride = default_row_index_stride; // Optional associated metadata const table_input_metadata* _metadata = nullptr; + // Optional footer key_value_metadata + std::map _user_data; friend chunked_orc_writer_options_builder; @@ -795,6 +826,11 @@ class chunked_orc_writer_options { */ table_input_metadata const* get_metadata() const { return _metadata; } + /** + * @brief Returns Key-Value footer metadata information. + */ + std::map const& get_key_value_metadata() const { return _user_data; } + // Setters /** @@ -849,6 +885,16 @@ class chunked_orc_writer_options { * @param meta Associated metadata. */ void metadata(table_input_metadata const* meta) { _metadata = meta; } + + /** + * @brief Sets Key-Value footer metadata. + * + * @param metadata Key-Value footer metadata + */ + void set_key_value_metadata(std::map metadata) + { + _user_data = std::move(metadata); + } }; class chunked_orc_writer_options_builder { @@ -941,6 +987,19 @@ class chunked_orc_writer_options_builder { return *this; } + /** + * @brief Sets Key-Value footer metadata. + * + * @param metadata Key-Value footer metadata + * @return this for chaining. + */ + chunked_orc_writer_options_builder& key_value_metadata( + std::map metadata) + { + options._user_data = std::move(metadata); + return *this; + } + /** * @brief move chunked_orc_writer_options member once it's built. */ diff --git a/cpp/include/cudf/io/parquet.hpp b/cpp/include/cudf/io/parquet.hpp index 61a1f76d986..d77ca6b3aea 100644 --- a/cpp/include/cudf/io/parquet.hpp +++ b/cpp/include/cudf/io/parquet.hpp @@ -398,6 +398,8 @@ class parquet_writer_options { std::vector _partitions; // Optional associated metadata table_input_metadata const* _metadata = nullptr; + // Optional footer key_value_metadata + std::vector> _user_data; // Parquet writer can write INT96 or TIMESTAMP_MICROS. Defaults to TIMESTAMP_MICROS. // If true then overrides any per-column setting in _metadata. bool _write_timestamps_as_int96 = false; @@ -476,6 +478,14 @@ class parquet_writer_options { */ table_input_metadata const* get_metadata() const { return _metadata; } + /** + * @brief Returns Key-Value footer metadata information. + */ + std::vector> const& get_key_value_metadata() const + { + return _user_data; + } + /** * @brief Returns `true` if timestamps will be written as INT96 */ @@ -517,6 +527,16 @@ class parquet_writer_options { */ void set_metadata(table_input_metadata const* metadata) { _metadata = metadata; } + /** + * @brief Sets metadata. + * + * @param metadata Key-Value footer metadata + */ + void set_key_value_metadata(std::vector> metadata) + { + _user_data = std::move(metadata); + } + /** * @brief Sets the level of statistics. * @@ -620,6 +640,19 @@ class parquet_writer_options_builder { return *this; } + /** + * @brief Sets Key-Value footer metadata in parquet_writer_options. + * + * @param metadata Key-Value footer metadata + * @return this for chaining. + */ + parquet_writer_options_builder& key_value_metadata( + std::vector> metadata) + { + options._user_data = std::move(metadata); + return *this; + } + /** * @brief Sets the level of statistics in parquet_writer_options. * @@ -757,6 +790,8 @@ class chunked_parquet_writer_options { statistics_freq _stats_level = statistics_freq::STATISTICS_ROWGROUP; // Optional associated metadata. table_input_metadata const* _metadata = nullptr; + // Optional footer key_value_metadata + std::vector> _user_data; // Parquet writer can write INT96 or TIMESTAMP_MICROS. Defaults to TIMESTAMP_MICROS. // If true then overrides any per-column setting in _metadata. bool _write_timestamps_as_int96 = false; @@ -802,6 +837,14 @@ class chunked_parquet_writer_options { */ table_input_metadata const* get_metadata() const { return _metadata; } + /** + * @brief Returns Key-Value footer metadata information. + */ + std::vector> const& get_key_value_metadata() const + { + return _user_data; + } + /** * @brief Returns `true` if timestamps will be written as INT96 */ @@ -824,6 +867,16 @@ class chunked_parquet_writer_options { */ void set_metadata(table_input_metadata const* metadata) { _metadata = metadata; } + /** + * @brief Sets Key-Value footer metadata. + * + * @param metadata Key-Value footer metadata + */ + void set_key_value_metadata(std::vector> metadata) + { + _user_data = std::move(metadata); + } + /** * @brief Sets the level of statistics in parquet_writer_options. * @@ -908,6 +961,19 @@ class chunked_parquet_writer_options_builder { return *this; } + /** + * @brief Sets Key-Value footer metadata in parquet_writer_options. + * + * @param metadata Key-Value footer metadata + * @return this for chaining. + */ + chunked_parquet_writer_options_builder& key_value_metadata( + std::vector> metadata) + { + options._user_data = std::move(metadata); + return *this; + } + /** * @brief Sets Sets the level of statistics in chunked_parquet_writer_options. * diff --git a/cpp/include/cudf/io/types.hpp b/cpp/include/cudf/io/types.hpp index 23533b82d0d..caaed259cfc 100644 --- a/cpp/include/cudf/io/types.hpp +++ b/cpp/include/cudf/io/types.hpp @@ -388,30 +388,7 @@ class table_input_metadata { */ table_input_metadata(table_view const& table); - /** - * @brief Construct a new table_input_metadata from a table_view. - * - * The constructed table_input_metadata has the same structure as the passed table_view - * - * @param table The table_view to construct metadata for - * @param user_data Additional metadata to encode, as key-value pairs - */ - table_input_metadata(table_view const& table, std::map user_data); - - /** - * @brief Construct a new table_input_metadata from a table_view. - * - * The constructed table_input_metadata has the same structure as the passed table_view - * - * @param table The table_view to construct metadata for - * @param user_data Additional metadata to encode, as key-value pairs. One per output file - */ - table_input_metadata(table_view const& table, - std::vector> user_data); - std::vector column_metadata; - std::vector> - user_data; //!< Format-dependent metadata as key-values pairs. One per output file }; struct partition_info { diff --git a/cpp/src/io/functions.cpp b/cpp/src/io/functions.cpp index 77bf733a501..121a1024561 100644 --- a/cpp/src/io/functions.cpp +++ b/cpp/src/io/functions.cpp @@ -429,20 +429,6 @@ std::unique_ptr> merge_row_group_metadata( } table_input_metadata::table_input_metadata(table_view const& table) - : table_input_metadata(table, std::map{}) -{ -} - -table_input_metadata::table_input_metadata(table_view const& table, - std::map user_data) - : table_input_metadata(table, - std::vector>{std::move(user_data)}) -{ -} - -table_input_metadata::table_input_metadata( - table_view const& table, std::vector> user_data) - : user_data{std::move(user_data)} { // Create a metadata hierarchy using `table` std::function get_children = [&](column_view const& col) { diff --git a/cpp/src/io/orc/writer_impl.cu b/cpp/src/io/orc/writer_impl.cu index fcdd0eff67e..6df26ca9e95 100644 --- a/cpp/src/io/orc/writer_impl.cu +++ b/cpp/src/io/orc/writer_impl.cu @@ -1311,6 +1311,7 @@ writer::impl::impl(std::unique_ptr sink, compression_kind_(to_orc_compression(options.get_compression())), enable_statistics_(options.is_enabled_statistics()), single_write_mode(mode == SingleWriteMode::YES), + kv_meta(options.get_key_value_metadata()), out_sink_(std::move(sink)) { if (options.get_metadata()) { @@ -1331,6 +1332,7 @@ writer::impl::impl(std::unique_ptr sink, compression_kind_(to_orc_compression(options.get_compression())), enable_statistics_(options.is_enabled_statistics()), single_write_mode(mode == SingleWriteMode::YES), + kv_meta(options.get_key_value_metadata()), out_sink_(std::move(sink)) { if (options.get_metadata()) { @@ -2067,15 +2069,10 @@ void writer::impl::close() PostScript ps; ff.contentLength = out_sink_->bytes_written(); - if (not table_meta->user_data.empty()) { - // ORC writer currently does not support multiple file writing - std::transform(table_meta->user_data[0].begin(), - table_meta->user_data[0].end(), - std::back_inserter(ff.metadata), - [&](auto const& udata) { - return UserMetadataItem{udata.first, udata.second}; - }); - } + std::transform( + kv_meta.begin(), kv_meta.end(), std::back_inserter(ff.metadata), [&](auto const& udata) { + return UserMetadataItem{udata.first, udata.second}; + }); // Write statistics metadata if (md.stripeStats.size() != 0) { diff --git a/cpp/src/io/orc/writer_impl.hpp b/cpp/src/io/orc/writer_impl.hpp index 89b8c305424..f43105d9638 100644 --- a/cpp/src/io/orc/writer_impl.hpp +++ b/cpp/src/io/orc/writer_impl.hpp @@ -369,6 +369,8 @@ class writer::impl { bool const single_write_mode; // optional user metadata std::unique_ptr table_meta; + // optional user metadata + std::map kv_meta; // to track if the output has been written to sink bool closed = false; diff --git a/cpp/src/io/parquet/writer_impl.cu b/cpp/src/io/parquet/writer_impl.cu index b4949aad04a..5666be3c6dd 100644 --- a/cpp/src/io/parquet/writer_impl.cu +++ b/cpp/src/io/parquet/writer_impl.cu @@ -1059,6 +1059,7 @@ writer::impl::impl(std::vector> sinks, compression_(to_parquet_compression(options.get_compression())), stats_granularity_(options.get_stats_level()), int96_timestamps(options.is_enabled_int96_timestamps()), + kv_md(options.get_key_value_metadata()), single_write_mode(mode == SingleWriteMode::YES), out_sink_(std::move(sinks)) { @@ -1080,6 +1081,7 @@ writer::impl::impl(std::vector> sinks, compression_(to_parquet_compression(options.get_compression())), stats_granularity_(options.get_stats_level()), int96_timestamps(options.is_enabled_int96_timestamps()), + kv_md(options.get_key_value_metadata()), single_write_mode(mode == SingleWriteMode::YES), out_sink_(std::move(sinks)) { @@ -1152,9 +1154,9 @@ void writer::impl::write(table_view const& table, std::vector pa md->column_order_listsize = (stats_granularity_ != statistics_freq::STATISTICS_NONE) ? num_columns : 0; - for (size_t p = 0; p < table_meta->user_data.size(); ++p) { - std::transform(table_meta->user_data[p].begin(), - table_meta->user_data[p].end(), + for (size_t p = 0; p < kv_md.size(); ++p) { + std::transform(kv_md[p].begin(), + kv_md[p].end(), std::back_inserter(md->files[p].key_value_metadata), [](auto const& kv) { return KeyValue{kv.first, kv.second}; diff --git a/cpp/src/io/parquet/writer_impl.hpp b/cpp/src/io/parquet/writer_impl.hpp index 36e587a9b6d..16e6d4f86e1 100644 --- a/cpp/src/io/parquet/writer_impl.hpp +++ b/cpp/src/io/parquet/writer_impl.hpp @@ -213,6 +213,8 @@ class writer::impl { bool int96_timestamps = false; // Overall file metadata. Filled in during the process and written during write_chunked_end() std::unique_ptr md; + // File footer key-value metadata. Written during write_chunked_end() + std::vector> kv_md; // optional user metadata std::unique_ptr table_meta; // to track if the output has been written to sink diff --git a/cpp/tests/io/parquet_test.cpp b/cpp/tests/io/parquet_test.cpp index 9ce79c7437c..59408bb78ed 100644 --- a/cpp/tests/io/parquet_test.cpp +++ b/cpp/tests/io/parquet_test.cpp @@ -834,13 +834,13 @@ TEST_F(ParquetWriterTest, MultiIndex) expected_metadata.column_metadata[2].set_name("int32s"); expected_metadata.column_metadata[3].set_name("floats"); expected_metadata.column_metadata[4].set_name("doubles"); - expected_metadata.user_data[0].insert( - {"pandas", "\"index_columns\": [\"int8s\", \"int16s\"], \"column1\": [\"int32s\"]"}); auto filepath = temp_env->get_temp_filepath("MultiIndex.parquet"); cudf_io::parquet_writer_options out_opts = cudf_io::parquet_writer_options::builder(cudf_io::sink_info{filepath}, expected->view()) - .metadata(&expected_metadata); + .metadata(&expected_metadata) + .key_value_metadata( + {{"pandas", "\"index_columns\": [\"int8s\", \"int16s\"], \"column1\": [\"int32s\"]"}}); cudf_io::write_parquet(out_opts); cudf_io::parquet_reader_options in_opts = From 9a77f5edd86962f68d27bccec38c3f35aae7624c Mon Sep 17 00:00:00 2001 From: Devavret Makkar Date: Thu, 9 Dec 2021 00:55:07 +0530 Subject: [PATCH 19/71] python changes for moving user_data --- python/cudf/cudf/_lib/cpp/io/orc.pxd | 9 +++++++++ python/cudf/cudf/_lib/cpp/io/parquet.pxd | 12 ++++++++++++ python/cudf/cudf/_lib/cpp/io/types.pxd | 5 ----- python/cudf/cudf/_lib/orc.pyx | 20 +++++++------------- python/cudf/cudf/_lib/parquet.pyx | 15 ++++++++------- 5 files changed, 36 insertions(+), 25 deletions(-) diff --git a/python/cudf/cudf/_lib/cpp/io/orc.pxd b/python/cudf/cudf/_lib/cpp/io/orc.pxd index 4b5ec913fb6..4eb014413b3 100644 --- a/python/cudf/cudf/_lib/cpp/io/orc.pxd +++ b/python/cudf/cudf/_lib/cpp/io/orc.pxd @@ -2,6 +2,7 @@ from libc.stdint cimport uint8_t from libcpp cimport bool +from libcpp.map cimport map from libcpp.memory cimport shared_ptr, unique_ptr from libcpp.string cimport string from libcpp.vector cimport vector @@ -87,6 +88,7 @@ cdef extern from "cudf/io/orc.hpp" \ void set_row_index_stride(size_type val) except+ void set_table(cudf_table_view.table_view tbl) except+ void set_metadata(cudf_io_types.table_input_metadata* meta) except+ + void set_key_value_metadata(map[string, string] kvm) except + @staticmethod orc_writer_options_builder builder( @@ -109,6 +111,9 @@ cdef extern from "cudf/io/orc.hpp" \ orc_writer_options_builder& metadata( cudf_io_types.table_input_metadata *meta ) except+ + orc_writer_options_builder& key_value_metadata( + map[string, string] kvm + ) except+ orc_writer_options build() except+ @@ -136,6 +141,7 @@ cdef extern from "cudf/io/orc.hpp" \ void set_metadata( cudf_io_types.table_input_metadata* meta ) except+ + void set_key_value_metadata(map[string, string] kvm) except + @staticmethod chunked_orc_writer_options_builder builder( @@ -157,6 +163,9 @@ cdef extern from "cudf/io/orc.hpp" \ chunked_orc_writer_options_builder& metadata( cudf_io_types.table_input_metadata *meta ) except+ + chunked_orc_writer_options_builder& key_value_metadata( + map[string, string] kvm + ) except+ chunked_orc_writer_options build() except+ diff --git a/python/cudf/cudf/_lib/cpp/io/parquet.pxd b/python/cudf/cudf/_lib/cpp/io/parquet.pxd index 9525ba37897..60be608d997 100644 --- a/python/cudf/cudf/_lib/cpp/io/parquet.pxd +++ b/python/cudf/cudf/_lib/cpp/io/parquet.pxd @@ -80,6 +80,9 @@ cdef extern from "cudf/io/parquet.hpp" namespace "cudf::io" nogil: void set_metadata( cudf_io_types.table_input_metadata *m ) except + + void set_key_value_metadata( + vector[map[string, string]] kvm + ) except + void set_stats_level( cudf_io_types.statistics_freq sf ) except + @@ -108,6 +111,9 @@ cdef extern from "cudf/io/parquet.hpp" namespace "cudf::io" nogil: parquet_writer_options_builder& metadata( cudf_io_types.table_input_metadata *m ) except + + parquet_writer_options_builder& key_value_metadata( + vector[map[string, string]] kvm + ) except + parquet_writer_options_builder& stats_level( cudf_io_types.statistics_freq sf ) except + @@ -146,6 +152,9 @@ cdef extern from "cudf/io/parquet.hpp" namespace "cudf::io" nogil: void set_metadata( cudf_io_types.table_input_metadata *m ) except + + void set_key_value_metadata( + vector[map[string, string]] kvm + ) except + void set_stats_level( cudf_io_types.statistics_freq sf ) except + @@ -168,6 +177,9 @@ cdef extern from "cudf/io/parquet.hpp" namespace "cudf::io" nogil: chunked_parquet_writer_options_builder& metadata( cudf_io_types.table_input_metadata *m ) except + + chunked_parquet_writer_options_builder& key_value_metadata( + vector[map[string, string]] kvm + ) except + chunked_parquet_writer_options_builder& stats_level( cudf_io_types.statistics_freq sf ) except + diff --git a/python/cudf/cudf/_lib/cpp/io/types.pxd b/python/cudf/cudf/_lib/cpp/io/types.pxd index 1db9a0e68f5..86c95b87fab 100644 --- a/python/cudf/cudf/_lib/cpp/io/types.pxd +++ b/python/cudf/cudf/_lib/cpp/io/types.pxd @@ -70,13 +70,8 @@ cdef extern from "cudf/io/types.hpp" \ cdef cppclass table_input_metadata: table_input_metadata() except + table_input_metadata(const cudf_table_view.table_view& table) except + - table_input_metadata( - const cudf_table_view.table_view& table, - map[string, string] user_data - ) except + vector[column_in_metadata] column_metadata - vector[map[string, string]] user_data cdef cppclass host_buffer: const char* data diff --git a/python/cudf/cudf/_lib/orc.pyx b/python/cudf/cudf/_lib/orc.pyx index 22547909528..ee9c35dc6de 100644 --- a/python/cudf/cudf/_lib/orc.pyx +++ b/python/cudf/cudf/_lib/orc.pyx @@ -311,22 +311,14 @@ cdef class ORCWriter: chunked_orc_writer_options anb creates a writer""" cdef table_view tv - # Set the table_metadata - cdef map[string, string] user_data - pandas_metadata = generate_pandas_metadata(table, self.index) - user_data[str.encode("pandas")] = str.encode(pandas_metadata) - num_index_cols_meta = 0 self.tbl_meta = make_unique[table_input_metadata]( table_view_from_table(table, ignore_index=True), - user_data, ) if self.index is not False: if isinstance(table._index, cudf.core.multiindex.MultiIndex): tv = table_view_from_table(table) - self.tbl_meta = make_unique[table_input_metadata]( - tv, user_data, - ) + self.tbl_meta = make_unique[table_input_metadata](tv) for level, idx_name in enumerate(table._index.names): self.tbl_meta.get().column_metadata[level].set_name( (str.encode(idx_name)) @@ -335,10 +327,7 @@ cdef class ORCWriter: else: if table._index.name is not None: tv = table_view_from_table(table) - self.tbl_meta = make_unique[table_input_metadata]( - tv, - user_data, - ) + self.tbl_meta = make_unique[table_input_metadata](tv) self.tbl_meta.get().column_metadata[0].set_name( str.encode(table._index.name) ) @@ -350,11 +339,16 @@ cdef class ORCWriter: table[name]._column, self.tbl_meta.get().column_metadata[i] ) + cdef map[string, string] user_data + pandas_metadata = generate_pandas_metadata(table, self.index) + user_data[str.encode("pandas")] = str.encode(pandas_metadata) + cdef chunked_orc_writer_options args with nogil: args = move( chunked_orc_writer_options.builder(self.sink) .metadata(self.tbl_meta.get()) + .key_value_metadata(move(user_data)) .compression(self.comp_type) .enable_statistics(self.enable_stats) .build() diff --git a/python/cudf/cudf/_lib/parquet.pyx b/python/cudf/cudf/_lib/parquet.pyx index e824c5152de..c99178fc343 100644 --- a/python/cudf/cudf/_lib/parquet.pyx +++ b/python/cudf/cudf/_lib/parquet.pyx @@ -296,7 +296,7 @@ cpdef write_parquet( # Create the write options cdef unique_ptr[table_input_metadata] tbl_meta - cdef map[string, string] user_data + cdef vector[map[string, string]] user_data cdef table_view tv cdef unique_ptr[cudf_io_types.data_sink] _data_sink cdef cudf_io_types.sink_info sink = make_sink_info(path, _data_sink) @@ -328,10 +328,8 @@ cpdef write_parquet( ) pandas_metadata = generate_pandas_metadata(table, index) - user_data[str.encode("pandas")] = str.encode(pandas_metadata) - - # Set the table_metadata - tbl_meta.get().user_data[0] = move(user_data) + user_data.resize(1) + user_data.back()[str.encode("pandas")] = str.encode(pandas_metadata) cdef cudf_io_types.compression_type comp_type = _get_comp_type(compression) cdef cudf_io_types.statistics_freq stat_freq = _get_stat_freq(statistics) @@ -346,6 +344,7 @@ cpdef write_parquet( cdef parquet_writer_options args = move( parquet_writer_options.builder(sink, tv) .metadata(tbl_meta.get()) + .key_value_metadata(move(user_data)) .compression(comp_type) .stats_level(stat_freq) .column_chunks_file_paths(c_column_chunks_file_paths) @@ -471,14 +470,16 @@ cdef class ParquetWriter: ) pandas_metadata = generate_pandas_metadata(table, self.index) - self.tbl_meta.get().user_data[0][str.encode("pandas")] = \ - str.encode(pandas_metadata) + cdef vector[map[string, string]] user_data + user_data.resize(1) + user_data.back()[str.encode("pandas")] = str.encode(pandas_metadata) cdef chunked_parquet_writer_options args with nogil: args = move( chunked_parquet_writer_options.builder(self.sink) .metadata(self.tbl_meta.get()) + .key_value_metadata(move(user_data)) .compression(self.comp_type) .stats_level(self.stat_freq) .build() From dc157e1ee18f1ce9369c4ac88740383eabc05101 Mon Sep 17 00:00:00 2001 From: Devavret Makkar Date: Thu, 9 Dec 2021 02:22:19 +0530 Subject: [PATCH 20/71] Add checks for sizes of options in case of multiple sinks --- cpp/include/cudf/io/parquet.hpp | 26 ++++++++++++++++++++++---- cpp/include/cudf/io/types.hpp | 12 ++++++------ cpp/src/io/functions.cpp | 2 +- 3 files changed, 29 insertions(+), 11 deletions(-) diff --git a/cpp/include/cudf/io/parquet.hpp b/cpp/include/cudf/io/parquet.hpp index d77ca6b3aea..4c500da2935 100644 --- a/cpp/include/cudf/io/parquet.hpp +++ b/cpp/include/cudf/io/parquet.hpp @@ -517,6 +517,8 @@ class parquet_writer_options { */ void set_partitions(std::vector const& partitions) { + CUDF_EXPECTS(partitions.size() == _sink.num_sinks, + "Mismatch between number of sinks and number of partitions"); _partitions = std::move(partitions); } @@ -534,6 +536,8 @@ class parquet_writer_options { */ void set_key_value_metadata(std::vector> metadata) { + CUDF_EXPECTS(metadata.size() == _sink.num_sinks, + "Mismatch between number of sinks and number of metadata maps"); _user_data = std::move(metadata); } @@ -562,11 +566,13 @@ class parquet_writer_options { /** * @brief Sets column chunks file path to be set in the raw output metadata. * - * @param file_paths Vector of Strings which indicates file path. Must be same size as partitions - * if partitions are specified + * @param file_paths Vector of Strings which indicates file path. Must be same size as number of + * data sinks in sink info */ void set_column_chunks_file_paths(std::vector file_paths) { + CUDF_EXPECTS(file_paths.size() == _sink.num_sinks, + "Mismatch between number of sinks and number of chunk paths to set"); _column_chunks_file_paths = std::move(file_paths); } @@ -624,6 +630,8 @@ class parquet_writer_options_builder { */ parquet_writer_options_builder& partitions(std::vector partitions) { + CUDF_EXPECTS(partitions.size() == options._sink.num_sinks, + "Mismatch between number of sinks and number of partitions"); options._partitions = std::move(partitions); return *this; } @@ -649,6 +657,8 @@ class parquet_writer_options_builder { parquet_writer_options_builder& key_value_metadata( std::vector> metadata) { + CUDF_EXPECTS(metadata.size() == options._sink.num_sinks, + "Mismatch between number of sinks and number of metadata maps"); options._user_data = std::move(metadata); return *this; } @@ -680,13 +690,15 @@ class parquet_writer_options_builder { /** * @brief Sets column chunks file path to be set in the raw output metadata. * - * @param file_paths Vector of Strings which indicates file path. Must be same size as partitions - * if partitions are specified + * @param file_paths Vector of Strings which indicates file path. Must be same size as number of + * data sinks * @return this for chaining. */ parquet_writer_options_builder& column_chunks_file_paths( std::vector const& file_paths) { + CUDF_EXPECTS(file_paths.size() == options._sink.num_sinks, + "Mismatch between number of sinks and number of chunk paths to set"); options._column_chunks_file_paths = file_paths; return *this; } @@ -874,6 +886,8 @@ class chunked_parquet_writer_options { */ void set_key_value_metadata(std::vector> metadata) { + CUDF_EXPECTS(metadata.size() == _sink.num_sinks, + "Mismatch between number of sinks and number of metadata maps"); _user_data = std::move(metadata); } @@ -970,6 +984,8 @@ class chunked_parquet_writer_options_builder { chunked_parquet_writer_options_builder& key_value_metadata( std::vector> metadata) { + CUDF_EXPECTS(metadata.size() == options._sink.num_sinks, + "Mismatch between number of sinks and number of metadata maps"); options._user_data = std::move(metadata); return *this; } @@ -1093,6 +1109,8 @@ class parquet_chunked_writer { * @param[in] table Table that needs to be written * @param[in] partitions Optional partitions to divide the table into. If specified, must be same * size as number of sinks. + * + * @throws cudf::logic_error If the number of partitions is not the smae as number of sinks * @return returns reference of the class object */ parquet_chunked_writer& write(table_view const& table, diff --git a/cpp/include/cudf/io/types.hpp b/cpp/include/cudf/io/types.hpp index caaed259cfc..7df4d5dde38 100644 --- a/cpp/include/cudf/io/types.hpp +++ b/cpp/include/cudf/io/types.hpp @@ -191,17 +191,17 @@ struct source_info { * @brief Destination information for write interfaces */ struct sink_info { - io_type type = io_type::VOID; - size_t num_void_sinks = 1; + io_type type = io_type::VOID; std::vector filepaths; std::vector*> buffers; // TODO: perhaps we can repurpose host_buffer. ask VM std::vector user_sinks; + size_t num_sinks = 1; sink_info() = default; - sink_info(size_t num_void_sinks) : type(io_type::VOID), num_void_sinks(num_void_sinks) {} + sink_info(size_t num_sinks) : type(io_type::VOID), num_sinks(num_sinks) {} explicit sink_info(std::vector const& file_paths) - : type(io_type::FILEPATH), filepaths(file_paths) + : type(io_type::FILEPATH), filepaths(file_paths), num_sinks(file_paths.size()) { } explicit sink_info(std::string const& file_path) : type(io_type::FILEPATH), filepaths({file_path}) @@ -209,13 +209,13 @@ struct sink_info { } explicit sink_info(std::vector*> const& buffers) - : type(io_type::HOST_BUFFER), buffers(buffers) + : type(io_type::HOST_BUFFER), buffers(buffers), num_sinks(buffers.size()) { } explicit sink_info(std::vector* buffer) : type(io_type::HOST_BUFFER), buffers({buffer}) {} explicit sink_info(std::vector const& user_sinks) - : type(io_type::USER_IMPLEMENTED), user_sinks(user_sinks) + : type(io_type::USER_IMPLEMENTED), user_sinks(user_sinks), num_sinks(user_sinks.size()) { } explicit sink_info(class cudf::io::data_sink* user_sink) diff --git a/cpp/src/io/functions.cpp b/cpp/src/io/functions.cpp index 121a1024561..b75176416df 100644 --- a/cpp/src/io/functions.cpp +++ b/cpp/src/io/functions.cpp @@ -132,7 +132,7 @@ std::vector> make_datasinks(sink_info const& info) case io_type::HOST_BUFFER: return cudf::io::data_sink::create(info.buffers); case io_type::VOID: { std::vector> sinks; - for (size_t i = 0; i < info.num_void_sinks; ++i) { + for (size_t i = 0; i < info.num_sinks; ++i) { sinks.push_back(cudf::io::data_sink::create()); } return sinks; From 8c2927d6b31c60b83cf269c25c92bb2218522f9b Mon Sep 17 00:00:00 2001 From: Devavret Makkar Date: Thu, 9 Dec 2021 02:40:01 +0530 Subject: [PATCH 21/71] bug in tests in init list for kv meta --- cpp/tests/io/parquet_test.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cpp/tests/io/parquet_test.cpp b/cpp/tests/io/parquet_test.cpp index 1346131da65..fb288f637d8 100644 --- a/cpp/tests/io/parquet_test.cpp +++ b/cpp/tests/io/parquet_test.cpp @@ -847,7 +847,7 @@ TEST_F(ParquetWriterTest, MultiIndex) cudf_io::parquet_writer_options::builder(cudf_io::sink_info{filepath}, expected->view()) .metadata(&expected_metadata) .key_value_metadata( - {{"pandas", "\"index_columns\": [\"int8s\", \"int16s\"], \"column1\": [\"int32s\"]"}}); + {{{"pandas", "\"index_columns\": [\"int8s\", \"int16s\"], \"column1\": [\"int32s\"]"}}}); cudf_io::write_parquet(out_opts); cudf_io::parquet_reader_options in_opts = From 200d1b0f65d89822d1f1a1016ad3ec73ca2b378f Mon Sep 17 00:00:00 2001 From: Devavret Makkar Date: Fri, 10 Dec 2021 03:04:34 +0530 Subject: [PATCH 22/71] Prevent setting chunk paths if not specified --- cpp/include/cudf/io/parquet.hpp | 11 +++++------ python/cudf/cudf/_lib/parquet.pyx | 6 +++--- 2 files changed, 8 insertions(+), 9 deletions(-) diff --git a/cpp/include/cudf/io/parquet.hpp b/cpp/include/cudf/io/parquet.hpp index 0eb65d389b4..2f96c0aab47 100644 --- a/cpp/include/cudf/io/parquet.hpp +++ b/cpp/include/cudf/io/parquet.hpp @@ -485,7 +485,7 @@ class parquet_writer_options { * @param partitions Partitions of input table in {start_row, num_rows} pairs. If specified, must * be same size as number of sinks in sink_info */ - void set_partitions(std::vector const& partitions) + void set_partitions(std::vector partitions) { CUDF_EXPECTS(partitions.size() == _sink.num_sinks, "Mismatch between number of sinks and number of partitions"); @@ -602,7 +602,7 @@ class parquet_writer_options_builder { { CUDF_EXPECTS(partitions.size() == options._sink.num_sinks, "Mismatch between number of sinks and number of partitions"); - options._partitions = std::move(partitions); + options.set_partitions(std::move(partitions)); return *this; } @@ -664,12 +664,11 @@ class parquet_writer_options_builder { * data sinks * @return this for chaining. */ - parquet_writer_options_builder& column_chunks_file_paths( - std::vector const& file_paths) + parquet_writer_options_builder& column_chunks_file_paths(std::vector file_paths) { CUDF_EXPECTS(file_paths.size() == options._sink.num_sinks, "Mismatch between number of sinks and number of chunk paths to set"); - options._column_chunks_file_paths = file_paths; + options.set_column_chunks_file_paths(std::move(file_paths)); return *this; } @@ -956,7 +955,7 @@ class chunked_parquet_writer_options_builder { { CUDF_EXPECTS(metadata.size() == options._sink.num_sinks, "Mismatch between number of sinks and number of metadata maps"); - options._user_data = std::move(metadata); + options.set_key_value_metadata(std::move(metadata)); return *this; } diff --git a/python/cudf/cudf/_lib/parquet.pyx b/python/cudf/cudf/_lib/parquet.pyx index c99178fc343..955324778fd 100644 --- a/python/cudf/cudf/_lib/parquet.pyx +++ b/python/cudf/cudf/_lib/parquet.pyx @@ -337,8 +337,6 @@ cpdef write_parquet( cdef unique_ptr[vector[uint8_t]] out_metadata_c cdef vector[string] c_column_chunks_file_paths cdef bool _int96_timestamps = int96_timestamps - if metadata_file_path is not None: - c_column_chunks_file_paths.push_back(str.encode(metadata_file_path)) # Perform write cdef parquet_writer_options args = move( @@ -347,10 +345,12 @@ cpdef write_parquet( .key_value_metadata(move(user_data)) .compression(comp_type) .stats_level(stat_freq) - .column_chunks_file_paths(c_column_chunks_file_paths) .int96_timestamps(_int96_timestamps) .build() ) + if metadata_file_path is not None: + c_column_chunks_file_paths.push_back(str.encode(metadata_file_path)) + args.set_column_chunks_file_paths(move(c_column_chunks_file_paths)) if row_group_size_bytes is not None: args.set_row_group_size_bytes(row_group_size_bytes) if row_group_size_rows is not None: From d0de9a9dbd7efdb70ad9d7d86fc67b55e63fb9e5 Mon Sep 17 00:00:00 2001 From: Devavret Makkar Date: Fri, 10 Dec 2021 04:00:21 +0530 Subject: [PATCH 23/71] Make returned metadata blob optional --- cpp/include/cudf/io/detail/parquet.hpp | 2 +- cpp/include/cudf/io/parquet.hpp | 4 ++-- cpp/include/cudf/io/types.hpp | 6 ++++++ cpp/src/io/functions.cpp | 6 +++--- cpp/src/io/parquet/writer_impl.cu | 14 +++++++------- cpp/src/io/parquet/writer_impl.hpp | 2 +- 6 files changed, 20 insertions(+), 14 deletions(-) diff --git a/cpp/include/cudf/io/detail/parquet.hpp b/cpp/include/cudf/io/detail/parquet.hpp index 561ce2d5b60..f1be72dc3d9 100644 --- a/cpp/include/cudf/io/detail/parquet.hpp +++ b/cpp/include/cudf/io/detail/parquet.hpp @@ -142,7 +142,7 @@ class writer { * @return A parquet-compatible blob that contains the data for all rowgroups in the list only if * `column_chunks_file_path` is provided, else null. */ - std::unique_ptr> close( + std::optional> close( std::vector const& column_chunks_file_path = {}); /** diff --git a/cpp/include/cudf/io/parquet.hpp b/cpp/include/cudf/io/parquet.hpp index 2f96c0aab47..0291e0d89cb 100644 --- a/cpp/include/cudf/io/parquet.hpp +++ b/cpp/include/cudf/io/parquet.hpp @@ -738,7 +738,7 @@ class parquet_writer_options_builder { * requested in parquet_writer_options (empty blob otherwise). */ -std::unique_ptr> write_parquet( +std::optional> write_parquet( parquet_writer_options const& options, rmm::mr::device_memory_resource* mr = rmm::mr::get_current_device_resource()); @@ -1093,7 +1093,7 @@ class parquet_chunked_writer { * @return A parquet-compatible blob that contains the data for all rowgroups in the list only if * `column_chunks_file_paths` is provided, else null. */ - std::unique_ptr> close( + std::optional> close( std::vector const& column_chunks_file_paths = {}); // Unique pointer to impl writer class diff --git a/cpp/include/cudf/io/types.hpp b/cpp/include/cudf/io/types.hpp index 7df4d5dde38..a81e6a4cb07 100644 --- a/cpp/include/cudf/io/types.hpp +++ b/cpp/include/cudf/io/types.hpp @@ -391,6 +391,12 @@ class table_input_metadata { std::vector column_metadata; }; +/** + * @brief Information used while writing partitioned datasets + * + * This information defines the slice of an input table to write to file. In partitioned dataset + * writing, one partition_info struct defines one partition and corresponds to one output file + */ struct partition_info { size_type start_row; size_type num_rows; diff --git a/cpp/src/io/functions.cpp b/cpp/src/io/functions.cpp index b75176416df..edd03762b1c 100644 --- a/cpp/src/io/functions.cpp +++ b/cpp/src/io/functions.cpp @@ -445,8 +445,8 @@ table_input_metadata::table_input_metadata(table_view const& table) /** * @copydoc cudf::io::write_parquet */ -std::unique_ptr> write_parquet(parquet_writer_options const& options, - rmm::mr::device_memory_resource* mr) +std::optional> write_parquet(parquet_writer_options const& options, + rmm::mr::device_memory_resource* mr) { namespace io_detail = cudf::io::detail; @@ -491,7 +491,7 @@ parquet_chunked_writer& parquet_chunked_writer::write(table_view const& table, /** * @copydoc cudf::io::parquet_chunked_writer::close */ -std::unique_ptr> parquet_chunked_writer::close( +std::optional> parquet_chunked_writer::close( std::vector const& column_chunks_file_path) { CUDF_FUNC_RANGE(); diff --git a/cpp/src/io/parquet/writer_impl.cu b/cpp/src/io/parquet/writer_impl.cu index f9872382f8d..6186c2bc39d 100644 --- a/cpp/src/io/parquet/writer_impl.cu +++ b/cpp/src/io/parquet/writer_impl.cu @@ -1564,12 +1564,12 @@ void writer::impl::write(table_view const& table, std::vector pa last_write_successful = true; } -std::unique_ptr> writer::impl::close( +std::optional> writer::impl::close( std::vector const& column_chunks_file_path) { - if (closed) { return nullptr; } + if (closed) { return std::nullopt; } closed = true; - if (not last_write_successful) { return nullptr; } + if (not last_write_successful) { return std::nullopt; } for (size_t p = 0; p < out_sink_.size(); p++) { std::vector buffer; CompactProtocolWriter cpw(&buffer); @@ -1616,11 +1616,11 @@ std::unique_ptr> writer::impl::close( buffer.insert(buffer.end(), reinterpret_cast(&fendr), reinterpret_cast(&fendr) + sizeof(fendr)); - return std::make_unique>(std::move(buffer)); + return buffer; } else { - return {nullptr}; + return std::nullopt; } - return nullptr; + return std::nullopt; } // Forward to implementation @@ -1652,7 +1652,7 @@ void writer::write(table_view const& table, std::vector const& p } // Forward to implementation -std::unique_ptr> writer::close( +std::optional> writer::close( std::vector const& column_chunks_file_path) { return _impl->close(column_chunks_file_path); diff --git a/cpp/src/io/parquet/writer_impl.hpp b/cpp/src/io/parquet/writer_impl.hpp index 16e6d4f86e1..06a8d02d99e 100644 --- a/cpp/src/io/parquet/writer_impl.hpp +++ b/cpp/src/io/parquet/writer_impl.hpp @@ -113,7 +113,7 @@ class writer::impl { * @return A parquet-compatible blob that contains the data for all rowgroups in the list only if * `column_chunks_file_path` is provided, else null. */ - std::unique_ptr> close( + std::optional> close( std::vector const& column_chunks_file_path = {}); private: From d90245f16fbaf0c7b500a21a07b6933b14dd9319 Mon Sep 17 00:00:00 2001 From: Devavret Makkar Date: Fri, 10 Dec 2021 04:14:43 +0530 Subject: [PATCH 24/71] Make sink info members private --- cpp/include/cudf/io/types.hpp | 23 ++++++++++++++--------- cpp/src/io/functions.cpp | 6 +++--- 2 files changed, 17 insertions(+), 12 deletions(-) diff --git a/cpp/include/cudf/io/types.hpp b/cpp/include/cudf/io/types.hpp index a81e6a4cb07..4463dc7ae8a 100644 --- a/cpp/include/cudf/io/types.hpp +++ b/cpp/include/cudf/io/types.hpp @@ -191,17 +191,11 @@ struct source_info { * @brief Destination information for write interfaces */ struct sink_info { - io_type type = io_type::VOID; - std::vector filepaths; - std::vector*> buffers; // TODO: perhaps we can repurpose host_buffer. ask VM - std::vector user_sinks; - size_t num_sinks = 1; - sink_info() = default; sink_info(size_t num_sinks) : type(io_type::VOID), num_sinks(num_sinks) {} explicit sink_info(std::vector const& file_paths) - : type(io_type::FILEPATH), filepaths(file_paths), num_sinks(file_paths.size()) + : type(io_type::FILEPATH), num_sinks(file_paths.size()), filepaths(file_paths) { } explicit sink_info(std::string const& file_path) : type(io_type::FILEPATH), filepaths({file_path}) @@ -209,19 +203,30 @@ struct sink_info { } explicit sink_info(std::vector*> const& buffers) - : type(io_type::HOST_BUFFER), buffers(buffers), num_sinks(buffers.size()) + : type(io_type::HOST_BUFFER), num_sinks(buffers.size()), buffers(buffers) { } explicit sink_info(std::vector* buffer) : type(io_type::HOST_BUFFER), buffers({buffer}) {} explicit sink_info(std::vector const& user_sinks) - : type(io_type::USER_IMPLEMENTED), user_sinks(user_sinks), num_sinks(user_sinks.size()) + : type(io_type::USER_IMPLEMENTED), num_sinks(user_sinks.size()), user_sinks(user_sinks) { } explicit sink_info(class cudf::io::data_sink* user_sink) : type(io_type::USER_IMPLEMENTED), user_sinks({user_sink}) { } + + auto const& get_filepaths() const { return filepaths; } + auto const& get_buffers() const { return buffers; } + auto const& get_user_sinks() const { return user_sinks; } + io_type type = io_type::VOID; + size_t num_sinks = 1; + + private: + std::vector filepaths; + std::vector*> buffers; // TODO: perhaps we can repurpose host_buffer. ask VM + std::vector user_sinks; }; class table_input_metadata; diff --git a/cpp/src/io/functions.cpp b/cpp/src/io/functions.cpp index edd03762b1c..c2591f50203 100644 --- a/cpp/src/io/functions.cpp +++ b/cpp/src/io/functions.cpp @@ -128,8 +128,8 @@ std::vector> make_datasources(source_info std::vector> make_datasinks(sink_info const& info) { switch (info.type) { - case io_type::FILEPATH: return cudf::io::data_sink::create(info.filepaths); - case io_type::HOST_BUFFER: return cudf::io::data_sink::create(info.buffers); + case io_type::FILEPATH: return cudf::io::data_sink::create(info.get_filepaths()); + case io_type::HOST_BUFFER: return cudf::io::data_sink::create(info.get_buffers()); case io_type::VOID: { std::vector> sinks; for (size_t i = 0; i < info.num_sinks; ++i) { @@ -137,7 +137,7 @@ std::vector> make_datasinks(sink_info const& info) } return sinks; } - case io_type::USER_IMPLEMENTED: return cudf::io::data_sink::create(info.user_sinks); + case io_type::USER_IMPLEMENTED: return cudf::io::data_sink::create(info.get_user_sinks()); default: CUDF_FAIL("Unsupported sink type"); } } From 5a17a4cfc0cca243e2fb2ddc65439303523572aa Mon Sep 17 00:00:00 2001 From: Devavret Makkar Date: Fri, 10 Dec 2021 05:20:01 +0530 Subject: [PATCH 25/71] Revert "Make returned metadata blob optional" This reverts commit d0de9a9dbd7efdb70ad9d7d86fc67b55e63fb9e5. --- cpp/include/cudf/io/detail/parquet.hpp | 2 +- cpp/include/cudf/io/parquet.hpp | 4 ++-- cpp/include/cudf/io/types.hpp | 6 ------ cpp/src/io/functions.cpp | 6 +++--- cpp/src/io/parquet/writer_impl.cu | 14 +++++++------- cpp/src/io/parquet/writer_impl.hpp | 2 +- 6 files changed, 14 insertions(+), 20 deletions(-) diff --git a/cpp/include/cudf/io/detail/parquet.hpp b/cpp/include/cudf/io/detail/parquet.hpp index f1be72dc3d9..561ce2d5b60 100644 --- a/cpp/include/cudf/io/detail/parquet.hpp +++ b/cpp/include/cudf/io/detail/parquet.hpp @@ -142,7 +142,7 @@ class writer { * @return A parquet-compatible blob that contains the data for all rowgroups in the list only if * `column_chunks_file_path` is provided, else null. */ - std::optional> close( + std::unique_ptr> close( std::vector const& column_chunks_file_path = {}); /** diff --git a/cpp/include/cudf/io/parquet.hpp b/cpp/include/cudf/io/parquet.hpp index 0291e0d89cb..2f96c0aab47 100644 --- a/cpp/include/cudf/io/parquet.hpp +++ b/cpp/include/cudf/io/parquet.hpp @@ -738,7 +738,7 @@ class parquet_writer_options_builder { * requested in parquet_writer_options (empty blob otherwise). */ -std::optional> write_parquet( +std::unique_ptr> write_parquet( parquet_writer_options const& options, rmm::mr::device_memory_resource* mr = rmm::mr::get_current_device_resource()); @@ -1093,7 +1093,7 @@ class parquet_chunked_writer { * @return A parquet-compatible blob that contains the data for all rowgroups in the list only if * `column_chunks_file_paths` is provided, else null. */ - std::optional> close( + std::unique_ptr> close( std::vector const& column_chunks_file_paths = {}); // Unique pointer to impl writer class diff --git a/cpp/include/cudf/io/types.hpp b/cpp/include/cudf/io/types.hpp index 4463dc7ae8a..c79c39b51be 100644 --- a/cpp/include/cudf/io/types.hpp +++ b/cpp/include/cudf/io/types.hpp @@ -396,12 +396,6 @@ class table_input_metadata { std::vector column_metadata; }; -/** - * @brief Information used while writing partitioned datasets - * - * This information defines the slice of an input table to write to file. In partitioned dataset - * writing, one partition_info struct defines one partition and corresponds to one output file - */ struct partition_info { size_type start_row; size_type num_rows; diff --git a/cpp/src/io/functions.cpp b/cpp/src/io/functions.cpp index c2591f50203..cd225673617 100644 --- a/cpp/src/io/functions.cpp +++ b/cpp/src/io/functions.cpp @@ -445,8 +445,8 @@ table_input_metadata::table_input_metadata(table_view const& table) /** * @copydoc cudf::io::write_parquet */ -std::optional> write_parquet(parquet_writer_options const& options, - rmm::mr::device_memory_resource* mr) +std::unique_ptr> write_parquet(parquet_writer_options const& options, + rmm::mr::device_memory_resource* mr) { namespace io_detail = cudf::io::detail; @@ -491,7 +491,7 @@ parquet_chunked_writer& parquet_chunked_writer::write(table_view const& table, /** * @copydoc cudf::io::parquet_chunked_writer::close */ -std::optional> parquet_chunked_writer::close( +std::unique_ptr> parquet_chunked_writer::close( std::vector const& column_chunks_file_path) { CUDF_FUNC_RANGE(); diff --git a/cpp/src/io/parquet/writer_impl.cu b/cpp/src/io/parquet/writer_impl.cu index 6186c2bc39d..f9872382f8d 100644 --- a/cpp/src/io/parquet/writer_impl.cu +++ b/cpp/src/io/parquet/writer_impl.cu @@ -1564,12 +1564,12 @@ void writer::impl::write(table_view const& table, std::vector pa last_write_successful = true; } -std::optional> writer::impl::close( +std::unique_ptr> writer::impl::close( std::vector const& column_chunks_file_path) { - if (closed) { return std::nullopt; } + if (closed) { return nullptr; } closed = true; - if (not last_write_successful) { return std::nullopt; } + if (not last_write_successful) { return nullptr; } for (size_t p = 0; p < out_sink_.size(); p++) { std::vector buffer; CompactProtocolWriter cpw(&buffer); @@ -1616,11 +1616,11 @@ std::optional> writer::impl::close( buffer.insert(buffer.end(), reinterpret_cast(&fendr), reinterpret_cast(&fendr) + sizeof(fendr)); - return buffer; + return std::make_unique>(std::move(buffer)); } else { - return std::nullopt; + return {nullptr}; } - return std::nullopt; + return nullptr; } // Forward to implementation @@ -1652,7 +1652,7 @@ void writer::write(table_view const& table, std::vector const& p } // Forward to implementation -std::optional> writer::close( +std::unique_ptr> writer::close( std::vector const& column_chunks_file_path) { return _impl->close(column_chunks_file_path); diff --git a/cpp/src/io/parquet/writer_impl.hpp b/cpp/src/io/parquet/writer_impl.hpp index 06a8d02d99e..16e6d4f86e1 100644 --- a/cpp/src/io/parquet/writer_impl.hpp +++ b/cpp/src/io/parquet/writer_impl.hpp @@ -113,7 +113,7 @@ class writer::impl { * @return A parquet-compatible blob that contains the data for all rowgroups in the list only if * `column_chunks_file_path` is provided, else null. */ - std::optional> close( + std::unique_ptr> close( std::vector const& column_chunks_file_path = {}); private: From 2e1c359925b829497448265c2530eec48f033db1 Mon Sep 17 00:00:00 2001 From: Devavret Makkar Date: Fri, 10 Dec 2021 16:34:26 +0530 Subject: [PATCH 26/71] make source data members private --- cpp/include/cudf/io/types.hpp | 19 ++++++++++++++++--- cpp/src/io/functions.cpp | 23 +++++++++++++---------- python/cudf/cudf/_lib/cpp/io/types.pxd | 10 +++++----- 3 files changed, 34 insertions(+), 18 deletions(-) diff --git a/cpp/include/cudf/io/types.hpp b/cpp/include/cudf/io/types.hpp index c79c39b51be..3c9e15d262b 100644 --- a/cpp/include/cudf/io/types.hpp +++ b/cpp/include/cudf/io/types.hpp @@ -152,10 +152,7 @@ struct host_buffer { */ struct source_info { io_type type = io_type::FILEPATH; - std::vector filepaths; - std::vector buffers; std::vector> files; - std::vector user_sources; source_info() = default; @@ -185,6 +182,16 @@ struct source_info { : type(io_type::USER_IMPLEMENTED), user_sources({source}) { } + + auto const& get_filepaths() const { return filepaths; } + auto const& get_buffers() const { return buffers; } + auto const& get_files() const { return files; } + auto const& get_user_sources() const { return user_sources; } + + private: + std::vector filepaths; + std::vector buffers; + std::vector user_sources; }; /** @@ -396,6 +403,12 @@ class table_input_metadata { std::vector column_metadata; }; +/** + * @brief Information used while writing partitioned datasets + * + * This information defines the slice of an input table to write to file. In partitioned dataset + * writing, one partition_info struct defines one partition and corresponds to one output file + */ struct partition_info { size_type start_row; size_type num_rows; diff --git a/cpp/src/io/functions.cpp b/cpp/src/io/functions.cpp index cd225673617..0944e6e3008 100644 --- a/cpp/src/io/functions.cpp +++ b/cpp/src/io/functions.cpp @@ -114,13 +114,13 @@ std::vector> make_datasources(source_info switch (info.type) { case io_type::FILEPATH: { auto sources = std::vector>(); - for (auto const& filepath : info.filepaths) { + for (auto const& filepath : info.get_filepaths()) { sources.emplace_back(cudf::io::datasource::create(filepath, range_offset, range_size)); } return sources; } - case io_type::HOST_BUFFER: return cudf::io::datasource::create(info.buffers); - case io_type::USER_IMPLEMENTED: return cudf::io::datasource::create(info.user_sources); + case io_type::HOST_BUFFER: return cudf::io::datasource::create(info.get_buffers()); + case io_type::USER_IMPLEMENTED: return cudf::io::datasource::create(info.get_user_sources()); default: CUDF_FAIL("Unsupported source type"); } } @@ -164,7 +164,7 @@ compression_type infer_compression_type(compression_type compression, source_inf if (info.type != io_type::FILEPATH) { return compression_type::NONE; } - auto filepath = info.filepaths[0]; + auto filepath = info.get_filepaths()[0]; // Attempt to infer from the file extension const auto pos = filepath.find_last_of('.'); @@ -243,14 +243,17 @@ raw_orc_statistics read_raw_orc_statistics(source_info const& src_info) // Get source to read statistics from std::unique_ptr source; if (src_info.type == io_type::FILEPATH) { - CUDF_EXPECTS(src_info.filepaths.size() == 1, "Only a single source is currently supported."); - source = cudf::io::datasource::create(src_info.filepaths[0]); + CUDF_EXPECTS(src_info.get_filepaths().size() == 1, + "Only a single source is currently supported."); + source = cudf::io::datasource::create(src_info.get_filepaths()[0]); } else if (src_info.type == io_type::HOST_BUFFER) { - CUDF_EXPECTS(src_info.buffers.size() == 1, "Only a single source is currently supported."); - source = cudf::io::datasource::create(src_info.buffers[0]); + CUDF_EXPECTS(src_info.get_buffers().size() == 1, + "Only a single source is currently supported."); + source = cudf::io::datasource::create(src_info.get_buffers()[0]); } else if (src_info.type == io_type::USER_IMPLEMENTED) { - CUDF_EXPECTS(src_info.user_sources.size() == 1, "Only a single source is currently supported."); - source = cudf::io::datasource::create(src_info.user_sources[0]); + CUDF_EXPECTS(src_info.get_user_sources().size() == 1, + "Only a single source is currently supported."); + source = cudf::io::datasource::create(src_info.get_user_sources()[0]); } else { CUDF_FAIL("Unsupported source type"); } diff --git a/python/cudf/cudf/_lib/cpp/io/types.pxd b/python/cudf/cudf/_lib/cpp/io/types.pxd index 86c95b87fab..1217f09f99c 100644 --- a/python/cudf/cudf/_lib/cpp/io/types.pxd +++ b/python/cudf/cudf/_lib/cpp/io/types.pxd @@ -82,8 +82,8 @@ cdef extern from "cudf/io/types.hpp" \ cdef cppclass source_info: io_type type - vector[string] filepaths - vector[host_buffer] buffers + const vector[string]& get_filepaths() except + + const vector[host_buffer]& get_buffers() except + vector[shared_ptr[CRandomAccessFile]] files source_info() except + @@ -93,9 +93,9 @@ cdef extern from "cudf/io/types.hpp" \ cdef cppclass sink_info: io_type type - string filepath - vector[char] * buffer - data_sink * user_sink + const vector[string]& get_filepaths() + const vector[vector[char] *]& get_buffers() + const vector[data_sink *]& get_user_sinks() sink_info() except + sink_info(string file_path) except + From f44a50b0bf723941a577bf6d71c3e00ba825d231 Mon Sep 17 00:00:00 2001 From: Devavret Makkar Date: Sat, 11 Dec 2021 00:15:24 +0530 Subject: [PATCH 27/71] Refactor aggregate_metadata --- cpp/src/io/parquet/writer_impl.cu | 117 ++++++++++++++++++----------- cpp/src/io/parquet/writer_impl.hpp | 4 +- 2 files changed, 74 insertions(+), 47 deletions(-) diff --git a/cpp/src/io/parquet/writer_impl.cu b/cpp/src/io/parquet/writer_impl.cu index f9872382f8d..3531fdb8eae 100644 --- a/cpp/src/io/parquet/writer_impl.cu +++ b/cpp/src/io/parquet/writer_impl.cu @@ -79,11 +79,43 @@ parquet::Compression to_parquet_compression(compression_type compression) } // namespace -struct per_file_metadata { - per_file_metadata(int num_files) : files(num_files) {} +struct aggregate_metadata { + aggregate_metadata(std::vector const& partitions, + size_type num_columns, + std::vector schema, + + statistics_freq stats_granularity, + std::vector> const& kv_md) + : version(1), schema(std::move(schema)), files(partitions.size()) + { + for (size_t i = 0; i < partitions.size(); ++i) { + this->files[i].num_rows = partitions[i].num_rows; + } + this->column_order_listsize = + (stats_granularity != statistics_freq::STATISTICS_NONE) ? num_columns : 0; + + for (size_t p = 0; p < kv_md.size(); ++p) { + std::transform(kv_md[p].begin(), + kv_md[p].end(), + std::back_inserter(this->files[p].key_value_metadata), + [](auto const& kv) { + return KeyValue{kv.first, kv.second}; + }); + } + } + + void update_files(std::vector const& partitions) + { + CUDF_EXPECTS(partitions.size() == this->files.size(), + "New partitions must be same size as previously passed number of partitions"); + for (size_t i = 0; i < partitions.size(); ++i) { + this->files[i].num_rows += partitions[i].num_rows; + } + } - FileMetaData get_metadata(int part) + FileMetaData get_metadata(size_t part) { + CUDF_EXPECTS(part < files.size(), "Invalid part index queried"); FileMetaData meta{}; meta.version = this->version; meta.schema = this->schema; @@ -95,14 +127,44 @@ struct per_file_metadata { return meta; } + void set_file_paths(std::vector const& column_chunks_file_path) + { + for (size_t p = 0; p < this->files.size(); ++p) { + auto& file = this->files[p]; + auto const& file_path = column_chunks_file_path[p]; + for (auto& rowgroup : file.row_groups) { + for (auto& col : rowgroup.columns) { + col.file_path = file_path; + } + } + } + } + + FileMetaData get_merged_metadata() + { + FileMetaData merged_md; + for (size_t p = 0; p < this->files.size(); ++p) { + auto& file = this->files[p]; + if (p == 0) { + merged_md = this->get_metadata(0); + } else { + merged_md.row_groups.insert(merged_md.row_groups.end(), + std::make_move_iterator(file.row_groups.begin()), + std::make_move_iterator(file.row_groups.end())); + merged_md.num_rows += file.num_rows; + } + } + return merged_md; + } + int32_t version = 0; std::vector schema; - struct per_file_members { + struct per_file_metadata { int64_t num_rows = 0; std::vector row_groups; std::vector key_value_metadata; }; - std::vector files; + std::vector files; std::string created_by = ""; uint32_t column_order_listsize = 0; }; @@ -1148,32 +1210,14 @@ void writer::impl::write(table_view const& table, std::vector pa std::vector this_table_schema(schema_tree.begin(), schema_tree.end()); if (!md) { - md = std::make_unique(partitions.size()); - md->version = 1; - for (size_t i = 0; i < partitions.size(); ++i) { - md->files[i].num_rows = partitions[i].num_rows; - } - md->column_order_listsize = - (stats_granularity_ != statistics_freq::STATISTICS_NONE) ? num_columns : 0; - - for (size_t p = 0; p < kv_md.size(); ++p) { - std::transform(kv_md[p].begin(), - kv_md[p].end(), - std::back_inserter(md->files[p].key_value_metadata), - [](auto const& kv) { - return KeyValue{kv.first, kv.second}; - }); - } - md->schema = this_table_schema; + md = std::make_unique( + partitions, num_columns, std::move(this_table_schema), stats_granularity_, kv_md); } else { // verify the user isn't passing mismatched tables CUDF_EXPECTS(md->schema == this_table_schema, "Mismatch in schema between multiple calls to write_chunk"); - // increment num rows - for (size_t i = 0; i < partitions.size(); ++i) { - md->files[i].num_rows += partitions[i].num_rows; - } + md->update_files(partitions); } // Create table_device_view so that corresponding column_device_view data // can be written into col_desc members @@ -1586,33 +1630,16 @@ std::unique_ptr> writer::impl::close( if (column_chunks_file_path.size() > 0) { CUDF_EXPECTS(column_chunks_file_path.size() == md->files.size(), "Expected one column chunk path per output file"); + md->set_file_paths(column_chunks_file_path); file_header_s fhdr = {parquet_magic}; std::vector buffer; CompactProtocolWriter cpw(&buffer); buffer.insert(buffer.end(), reinterpret_cast(&fhdr), reinterpret_cast(&fhdr) + sizeof(fhdr)); - FileMetaData merged_md; - for (size_t p = 0; p < md->files.size(); ++p) { - auto& file = md->files[p]; - auto const& file_path = column_chunks_file_path[p]; - for (auto& rowgroup : file.row_groups) { - for (auto& col : rowgroup.columns) { - col.file_path = file_path; - } - } - if (p == 0) { - merged_md = md->get_metadata(0); - } else { - merged_md.row_groups.insert(merged_md.row_groups.end(), - std::make_move_iterator(file.row_groups.begin()), - std::make_move_iterator(file.row_groups.end())); - merged_md.num_rows += file.num_rows; - } - } file_ender_s fendr; fendr.magic = parquet_magic; - fendr.footer_len = static_cast(cpw.write(merged_md)); + fendr.footer_len = static_cast(cpw.write(md->get_merged_metadata())); buffer.insert(buffer.end(), reinterpret_cast(&fendr), reinterpret_cast(&fendr) + sizeof(fendr)); diff --git a/cpp/src/io/parquet/writer_impl.hpp b/cpp/src/io/parquet/writer_impl.hpp index 16e6d4f86e1..79de9d92e2f 100644 --- a/cpp/src/io/parquet/writer_impl.hpp +++ b/cpp/src/io/parquet/writer_impl.hpp @@ -45,7 +45,7 @@ namespace detail { namespace parquet { // Forward internal classes struct parquet_column_view; -struct per_file_metadata; +struct aggregate_metadata; using namespace cudf::io::parquet; using namespace cudf::io; @@ -212,7 +212,7 @@ class writer::impl { statistics_freq stats_granularity_ = statistics_freq::STATISTICS_NONE; bool int96_timestamps = false; // Overall file metadata. Filled in during the process and written during write_chunked_end() - std::unique_ptr md; + std::unique_ptr md; // File footer key-value metadata. Written during write_chunked_end() std::vector> kv_md; // optional user metadata From be8c19a1161dbe99d821dea26bf37f3e527985af Mon Sep 17 00:00:00 2001 From: Devavret Makkar Date: Sat, 11 Dec 2021 00:19:53 +0530 Subject: [PATCH 28/71] revert tests that were changed for debugging --- cpp/tests/io/parquet_test.cpp | 34 ++++++++++++++-------------------- 1 file changed, 14 insertions(+), 20 deletions(-) diff --git a/cpp/tests/io/parquet_test.cpp b/cpp/tests/io/parquet_test.cpp index fb288f637d8..feb9235741a 100644 --- a/cpp/tests/io/parquet_test.cpp +++ b/cpp/tests/io/parquet_test.cpp @@ -580,21 +580,18 @@ TEST_F(ParquetWriterTest, Strings) expected_metadata.column_metadata[1].set_name("col_string"); expected_metadata.column_metadata[2].set_name("col_another"); + auto filepath = temp_env->get_temp_filepath("Strings.parquet"); cudf_io::parquet_writer_options out_opts = - cudf_io::parquet_writer_options::builder( - cudf_io::sink_info(std::vector{"first.parquet", "second.parquet"}), - expected->view()) - .partitions({{1, 2}, {4, 1}}) - .metadata(&expected_metadata) - .stats_level(cudf::io::STATISTICS_NONE); + cudf_io::parquet_writer_options::builder(cudf_io::sink_info{filepath}, expected->view()) + .metadata(&expected_metadata); cudf_io::write_parquet(out_opts); - // cudf_io::parquet_reader_options in_opts = - // cudf_io::parquet_reader_options::builder(cudf_io::source_info{filepath}); - // auto result = cudf_io::read_parquet(in_opts); + cudf_io::parquet_reader_options in_opts = + cudf_io::parquet_reader_options::builder(cudf_io::source_info{filepath}); + auto result = cudf_io::read_parquet(in_opts); - // CUDF_TEST_EXPECT_TABLES_EQUAL(expected->view(), result.tbl->view()); - // cudf::test::expect_metadata_equal(expected_metadata, result.metadata); + CUDF_TEST_EXPECT_TABLES_EQUAL(expected->view(), result.tbl->view()); + cudf::test::expect_metadata_equal(expected_metadata, result.metadata); } TEST_F(ParquetWriterTest, SlicedTable) @@ -2155,20 +2152,17 @@ TEST_F(ParquetWriterStressTest, LargeTableGoodCompression) // exercises multiple rowgroups srand(31337); - auto expected = create_compressible_fixed_table(16, 4 * 1024 * 1024, 1000, false); + auto expected = create_compressible_fixed_table(16, 4 * 1024 * 1024, 128 * 1024, false); // write out using the custom sink (which uses device writes) cudf_io::parquet_writer_options args = - cudf_io::parquet_writer_options::builder( - cudf_io::sink_info(std::vector{"first.parquet", "second.parquet"}), *expected) - .partitions({{10, 20 * 1024}, {20 * 1024 + 7, 30 * 1024}}) - .compression(cudf_io::compression_type::NONE); + cudf_io::parquet_writer_options::builder(cudf_io::sink_info{&custom_sink}, *expected); cudf_io::write_parquet(args); - // cudf_io::parquet_reader_options custom_args = - // cudf_io::parquet_reader_options::builder(cudf_io::source_info{mm_buf.data(), mm_buf.size()}); - // auto custom_tbl = cudf_io::read_parquet(custom_args); - // CUDF_TEST_EXPECT_TABLES_EQUAL(custom_tbl.tbl->view(), expected->view()); + cudf_io::parquet_reader_options custom_args = + cudf_io::parquet_reader_options::builder(cudf_io::source_info{mm_buf.data(), mm_buf.size()}); + auto custom_tbl = cudf_io::read_parquet(custom_args); + CUDF_TEST_EXPECT_TABLES_EQUAL(custom_tbl.tbl->view(), expected->view()); } TEST_F(ParquetWriterStressTest, LargeTableWithValids) From b9b5c15c42b6856f0c3eb44afc0ff438f67e0078 Mon Sep 17 00:00:00 2001 From: Devavret Makkar Date: Sat, 11 Dec 2021 02:41:00 +0530 Subject: [PATCH 29/71] Add empty df tests, make review changes --- cpp/include/cudf/io/types.hpp | 2 +- cpp/src/io/parquet/writer_impl.cu | 16 +++----- cpp/src/io/parquet/writer_impl.hpp | 4 +- cpp/tests/io/parquet_test.cpp | 62 ++++++++++++++++++++++++++++++ 4 files changed, 72 insertions(+), 12 deletions(-) diff --git a/cpp/include/cudf/io/types.hpp b/cpp/include/cudf/io/types.hpp index 3c9e15d262b..9820eeaf977 100644 --- a/cpp/include/cudf/io/types.hpp +++ b/cpp/include/cudf/io/types.hpp @@ -232,7 +232,7 @@ struct sink_info { private: std::vector filepaths; - std::vector*> buffers; // TODO: perhaps we can repurpose host_buffer. ask VM + std::vector*> buffers; std::vector user_sinks; }; diff --git a/cpp/src/io/parquet/writer_impl.cu b/cpp/src/io/parquet/writer_impl.cu index 3531fdb8eae..b266a343791 100644 --- a/cpp/src/io/parquet/writer_impl.cu +++ b/cpp/src/io/parquet/writer_impl.cu @@ -1169,11 +1169,10 @@ void writer::impl::init_state() } } -void writer::impl::write(table_view const& table, std::vector partitions) +void writer::impl::write(table_view const& table, std::vector const& partitions) { last_write_successful = false; CUDF_EXPECTS(not closed, "Data has already been flushed to out and closed"); - if (partitions.empty()) { partitions.push_back({0, table.num_rows()}); } if (not table_meta) { table_meta = std::make_unique(table); } @@ -1419,14 +1418,10 @@ void writer::impl::write(table_view const& table, std::vector pa } // Find which partition a rg belongs to - std::vector part_end_rg; - std::inclusive_scan( - num_rg_in_part.begin(), num_rg_in_part.end(), std::back_inserter(part_end_rg)); std::vector rg_to_part; - auto it = thrust::make_counting_iterator(0); - std::transform(it, it + num_rowgroups, std::back_inserter(rg_to_part), [&](auto i) { - return std::upper_bound(part_end_rg.begin(), part_end_rg.end(), i) - part_end_rg.begin(); - }); + for (size_t p = 0; p < num_rg_in_part.size(); ++p) { + std::fill_n(std::back_inserter(rg_to_part), num_rg_in_part[p], p); + } // Initialize batches of rowgroups to encode (mainly to limit peak memory usage) std::vector batch_list; @@ -1675,7 +1670,8 @@ writer::~writer() = default; // Forward to implementation void writer::write(table_view const& table, std::vector const& partitions) { - _impl->write(table, partitions); + _impl->write( + table, partitions.empty() ? std::vector{{0, table.num_rows()}} : partitions); } // Forward to implementation diff --git a/cpp/src/io/parquet/writer_impl.hpp b/cpp/src/io/parquet/writer_impl.hpp index 79de9d92e2f..10537d72513 100644 --- a/cpp/src/io/parquet/writer_impl.hpp +++ b/cpp/src/io/parquet/writer_impl.hpp @@ -103,8 +103,10 @@ class writer::impl { * normally used for chunked writing. * * @param[in] table The table information to be written + * @param[in] partitions Optional partitions to divide the table into. If specified, must be same + * size as number of sinks. */ - void write(table_view const& table, std::vector partitions); + void write(table_view const& table, std::vector const& partitions); /** * @brief Finishes the chunked/streamed write process. diff --git a/cpp/tests/io/parquet_test.cpp b/cpp/tests/io/parquet_test.cpp index feb9235741a..75ff39cbe70 100644 --- a/cpp/tests/io/parquet_test.cpp +++ b/cpp/tests/io/parquet_test.cpp @@ -1206,6 +1206,68 @@ TEST_F(ParquetWriterTest, PartitionedWrite) CUDF_TEST_EXPECT_TABLES_EQUAL(expected2, result2.tbl->view()); } +TEST_F(ParquetWriterTest, PartitionedWriteEmptyPartitions) +{ + auto source = create_random_fixed_table(4, 4, false); + + auto filepath1 = temp_env->get_temp_filepath("PartitionedWrite1.parquet"); + auto filepath2 = temp_env->get_temp_filepath("PartitionedWrite2.parquet"); + + auto partition1 = cudf::io::partition_info{1, 0}; + auto partition2 = cudf::io::partition_info{1, 0}; + + auto expected1 = + cudf::slice(*source, {partition1.start_row, partition1.start_row + partition1.num_rows}); + auto expected2 = + cudf::slice(*source, {partition2.start_row, partition2.start_row + partition2.num_rows}); + + cudf_io::parquet_writer_options args = + cudf_io::parquet_writer_options::builder( + cudf_io::sink_info(std::vector{filepath1, filepath2}), *source) + .partitions({partition1, partition2}) + .compression(cudf_io::compression_type::NONE); + cudf_io::write_parquet(args); + + auto result1 = cudf_io::read_parquet( + cudf_io::parquet_reader_options::builder(cudf_io::source_info(filepath1))); + CUDF_TEST_EXPECT_TABLES_EQUAL(expected1, result1.tbl->view()); + + auto result2 = cudf_io::read_parquet( + cudf_io::parquet_reader_options::builder(cudf_io::source_info(filepath2))); + CUDF_TEST_EXPECT_TABLES_EQUAL(expected2, result2.tbl->view()); +} + +TEST_F(ParquetWriterTest, PartitionedWriteEmptyColumns) +{ + auto source = create_random_fixed_table(0, 4, false); + + auto filepath1 = temp_env->get_temp_filepath("PartitionedWrite1.parquet"); + auto filepath2 = temp_env->get_temp_filepath("PartitionedWrite2.parquet"); + + auto partition1 = cudf::io::partition_info{1, 0}; + auto partition2 = cudf::io::partition_info{1, 0}; + + auto expected1 = + cudf::slice(*source, {partition1.start_row, partition1.start_row + partition1.num_rows}); + auto expected2 = + cudf::slice(*source, {partition2.start_row, partition2.start_row + partition2.num_rows}); + + cudf_io::parquet_writer_options args = + cudf_io::parquet_writer_options::builder( + cudf_io::sink_info(std::vector{filepath1, filepath2}), *source) + .partitions({partition1, partition2}) + .compression(cudf_io::compression_type::NONE); + cudf_io::write_parquet(args); + + auto result1 = cudf_io::read_parquet( + cudf_io::parquet_reader_options::builder(cudf_io::source_info(filepath1))); + CUDF_TEST_EXPECT_TABLES_EQUAL(expected1, result1.tbl->view()); + + auto result2 = cudf_io::read_parquet( + cudf_io::parquet_reader_options::builder(cudf_io::source_info(filepath2))); + CUDF_TEST_EXPECT_TABLES_EQUAL(expected2, result2.tbl->view()); +} + template std::string create_parquet_file(int num_cols) { From 85ce42fcbe9cb5f0914254e0edef6d426fee1e66 Mon Sep 17 00:00:00 2001 From: Devavret Makkar Date: Sat, 11 Dec 2021 05:25:46 +0530 Subject: [PATCH 30/71] Initial somewhat working python bindings for parquet partitioned writing --- cpp/include/cudf/io/types.hpp | 5 + python/cudf/cudf/_lib/cpp/io/parquet.pxd | 3 + python/cudf/cudf/_lib/cpp/io/types.pxd | 8 + python/cudf/cudf/_lib/io/utils.pxd | 1 + python/cudf/cudf/_lib/io/utils.pyx | 29 ++++ python/cudf/cudf/_lib/parquet.pyx | 17 +- python/cudf/cudf/io/parquet.py | 204 ++++++++++++++++------- 7 files changed, 200 insertions(+), 67 deletions(-) diff --git a/cpp/include/cudf/io/types.hpp b/cpp/include/cudf/io/types.hpp index 9820eeaf977..4dc65ee9648 100644 --- a/cpp/include/cudf/io/types.hpp +++ b/cpp/include/cudf/io/types.hpp @@ -412,6 +412,11 @@ class table_input_metadata { struct partition_info { size_type start_row; size_type num_rows; + + partition_info() = default; + partition_info(size_type start_row, size_type num_rows) : start_row(start_row), num_rows(num_rows) + { + } }; } // namespace io diff --git a/python/cudf/cudf/_lib/cpp/io/parquet.pxd b/python/cudf/cudf/_lib/cpp/io/parquet.pxd index 60be608d997..338ac0d6f88 100644 --- a/python/cudf/cudf/_lib/cpp/io/parquet.pxd +++ b/python/cudf/cudf/_lib/cpp/io/parquet.pxd @@ -108,6 +108,9 @@ cdef extern from "cudf/io/parquet.hpp" namespace "cudf::io" nogil: cudf_io_types.sink_info sink_, cudf_table_view.table_view table_ ) except + + parquet_writer_options_builder& partitions( + vector[cudf_io_types.partition_info] partitions + ) except + parquet_writer_options_builder& metadata( cudf_io_types.table_input_metadata *m ) except + diff --git a/python/cudf/cudf/_lib/cpp/io/types.pxd b/python/cudf/cudf/_lib/cpp/io/types.pxd index 1217f09f99c..4bca9e2a100 100644 --- a/python/cudf/cudf/_lib/cpp/io/types.pxd +++ b/python/cudf/cudf/_lib/cpp/io/types.pxd @@ -73,6 +73,13 @@ cdef extern from "cudf/io/types.hpp" \ vector[column_in_metadata] column_metadata + cdef cppclass partition_info: + size_type start_row + size_type num_rows + + partition_info() + partition_info(size_type start_row, size_type num_rows) except + + cdef cppclass host_buffer: const char* data size_t size @@ -99,6 +106,7 @@ cdef extern from "cudf/io/types.hpp" \ sink_info() except + sink_info(string file_path) except + + sink_info(vector[string] file_path) except + sink_info(vector[char] * buffer) except + sink_info(data_sink * user_sink) except + diff --git a/python/cudf/cudf/_lib/io/utils.pxd b/python/cudf/cudf/_lib/io/utils.pxd index 36520538506..1c0926b90c6 100644 --- a/python/cudf/cudf/_lib/io/utils.pxd +++ b/python/cudf/cudf/_lib/io/utils.pxd @@ -13,6 +13,7 @@ from cudf._lib.cpp.io.types cimport ( cdef source_info make_source_info(list src) except* +cdef sink_info make_sinks_info(list src, vector[unique_ptr[data_sink]] & data) except* cdef sink_info make_sink_info(src, unique_ptr[data_sink] & data) except* cdef update_struct_field_names( table, diff --git a/python/cudf/cudf/_lib/io/utils.pyx b/python/cudf/cudf/_lib/io/utils.pyx index 8e07ee92fde..4a902165bac 100644 --- a/python/cudf/cudf/_lib/io/utils.pyx +++ b/python/cudf/cudf/_lib/io/utils.pyx @@ -79,6 +79,35 @@ cdef source_info make_source_info(list src) except*: return source_info(c_host_buffers) # Converts the Python sink input to libcudf++ IO sink_info. +cdef sink_info make_sinks_info(list src, vector[unique_ptr[data_sink]] & sink) except*: + cdef vector[data_sink] data_sinks + # TODO: re-enable after paths splits is known to work + # if isinstance(src[0], io.StringIO): + # for s in src: + # sink.push_back(new iobase_data_sink(src)) + # data_sinks.push_back(sink.back().get()) + # return sink_info(data_sinks) + # elif isinstance(src[0], io.TextIOBase): + # # Files opened in text mode expect writes to be str rather than bytes, + # # which requires conversion from utf-8. If the underlying buffer is + # # utf-8, we can bypass this conversion by writing directly to it. + # if codecs.lookup(src.encoding).name not in {"utf-8", "ascii"}: + # raise NotImplementedError(f"Unsupported encoding {src.encoding}") + # sink.reset(new iobase_data_sink(src.buffer)) + # return sink_info(sink.get()) + # if isinstance(src[0], io.IOBase): + # for s in src: + # data_sinks.push_back(new iobase_data_sink(s)) + # return sink_info(sink.get()) + # elif isinstance(src[0], (basestring, os.PathLike)): + cdef vector[string] paths + for s in src: + paths.push_back( os.path.expanduser(s).encode()) + return sink_info(move(paths)) + # else: + # raise TypeError("Unrecognized input type: {}".format(type(src))) + + cdef sink_info make_sink_info(src, unique_ptr[data_sink] & sink) except*: if isinstance(src, io.StringIO): sink.reset(new iobase_data_sink(src)) diff --git a/python/cudf/cudf/_lib/parquet.pyx b/python/cudf/cudf/_lib/parquet.pyx index 955324778fd..380165b9188 100644 --- a/python/cudf/cudf/_lib/parquet.pyx +++ b/python/cudf/cudf/_lib/parquet.pyx @@ -60,6 +60,7 @@ from cudf._lib.cpp.types cimport data_type, size_type from cudf._lib.io.datasource cimport Datasource, NativeFileDatasource from cudf._lib.io.utils cimport ( make_sink_info, + make_sinks_info, make_source_info, update_struct_field_names, ) @@ -277,14 +278,15 @@ cpdef read_parquet(filepaths_or_buffers, columns=None, row_groups=None, cpdef write_parquet( table, - object path, + object filepaths_or_buffers, object index=None, object compression="snappy", object statistics="ROWGROUP", object metadata_file_path=None, object int96_timestamps=False, object row_group_size_bytes=None, - object row_group_size_rows=None): + object row_group_size_rows=None, + object partitions_info=None): """ Cython function to call into libcudf API, see `write_parquet`. @@ -298,8 +300,8 @@ cpdef write_parquet( cdef vector[map[string, string]] user_data cdef table_view tv - cdef unique_ptr[cudf_io_types.data_sink] _data_sink - cdef cudf_io_types.sink_info sink = make_sink_info(path, _data_sink) + cdef vector[unique_ptr[cudf_io_types.data_sink]] _data_sinks + cdef cudf_io_types.sink_info sink = make_sinks_info(filepaths_or_buffers, _data_sinks) if index is True or ( index is None and not isinstance(table._index, cudf.RangeIndex) @@ -337,12 +339,17 @@ cpdef write_parquet( cdef unique_ptr[vector[uint8_t]] out_metadata_c cdef vector[string] c_column_chunks_file_paths cdef bool _int96_timestamps = int96_timestamps + cdef vector[cudf_io_types.partition_info] partitions + if partitions_info is not None: + for part in partitions_info: + partitions.push_back(cudf_io_types.partition_info(part[0], part[1])) # Perform write cdef parquet_writer_options args = move( parquet_writer_options.builder(sink, tv) + .partitions(partitions) # move if possible .metadata(tbl_meta.get()) - .key_value_metadata(move(user_data)) + # .key_value_metadata(move(user_data)) .compression(comp_type) .stats_level(stat_freq) .int96_timestamps(_int96_timestamps) diff --git a/python/cudf/cudf/io/parquet.py b/python/cudf/cudf/io/parquet.py index f9b39bf2cfa..62461e3fd37 100644 --- a/python/cudf/cudf/io/parquet.py +++ b/python/cudf/cudf/io/parquet.py @@ -2,8 +2,10 @@ import io import json +from typing import ValuesView import warnings from collections import defaultdict +from contextlib import ExitStack from uuid import uuid4 import fsspec @@ -17,6 +19,69 @@ from cudf.utils import ioutils +def _write_parquet( + df, + paths, + compression="snappy", + index=None, + statistics="ROWGROUP", + metadata_file_path=None, + int96_timestamps=False, + row_group_size_bytes=None, + row_group_size_rows=None, + partitions_info=None, + **kwargs, +): + # TODO: exceptions: + # If paths are multiple then partitions_info are required. + # If partitions_info are specified then paths should be list like. + # In either case, both should be of same length. + # Partition offsets should be a list of 2-tuples + if is_list_like(paths) or partitions_info is not None: + if not len(paths) == len(partitions_info): + ValueError("") + + paths_or_bufs = [ + ioutils.get_writer_filepath_or_buffer(path, mode="wb", **kwargs) + for path in paths + ] + if all([ioutils.is_fsspec_open_file(buf) for buf in paths_or_bufs]): + with ExitStack() as stack: + fsspec_objs = [ + stack.enter_context(open(file)) for file in paths_or_bufs + ] + file_objs = [ + ioutils.get_IOBase_writer(file_obj) for file_obj in fsspec_objs + ] + write_parquet_res = libparquet.write_parquet( + df, + filepaths_or_buffers=file_objs, + index=index, + compression=compression, + statistics=statistics, + metadata_file_path=metadata_file_path, + int96_timestamps=int96_timestamps, + row_group_size_bytes=row_group_size_bytes, + row_group_size_rows=row_group_size_rows, + partitions_info=partitions_info, + ) + else: + write_parquet_res = libparquet.write_parquet( + df, + filepaths_or_buffers=paths_or_bufs, + index=index, + compression=compression, + statistics=statistics, + metadata_file_path=metadata_file_path, + int96_timestamps=int96_timestamps, + row_group_size_bytes=row_group_size_bytes, + row_group_size_rows=row_group_size_rows, + partitions_info=partitions_info, + ) + + return write_parquet_res + + def _get_partition_groups(df, partition_cols, preserve_index=False): # TODO: We can use groupby functionality here after cudf#4346. # Longer term, we want more slicing logic to be pushed down @@ -28,10 +93,13 @@ def _get_partition_groups(df, partition_cols, preserve_index=False): divisions = df[partition_cols].drop_duplicates(ignore_index=True) splits = df[partition_cols].searchsorted(divisions, side="left") splits = splits.tolist() + [len(df[partition_cols])] - return [ - df.iloc[splits[i] : splits[i + 1]].copy(deep=False) - for i in range(0, len(splits) - 1) - ] + return ( + splits, + [ + df.iloc[splits[i] : splits[i + 1]].copy(deep=False) + for i in range(0, len(splits) - 1) + ], + ) # Logic chosen to match: https://arrow.apache.org/ @@ -93,13 +161,23 @@ def write_to_dataset( raise ValueError("No data left to save outside partition columns") # Loop through the partition groups - for _, sub_df in enumerate( - _get_partition_groups( - df, partition_cols, preserve_index=preserve_index + partition_offsets, partition_dfs = _get_partition_groups( + df, partition_cols, preserve_index=preserve_index + ) + partitions_info = [ + ( + partition_offsets[i], + partition_offsets[i + 1] - partition_offsets[i], ) - ): + for i in range(0, len(partition_offsets) - 1) + ] + + full_paths = [] + for _, sub_df in enumerate(partition_dfs): if sub_df is None or len(sub_df) == 0: continue + # TODO: This will cause multiple device reads. When we use groupby, + # we get a keys column which we can transfer in a single d->h copy keys = tuple([sub_df[col].iloc[0] for col in partition_cols]) if not isinstance(keys, tuple): keys = (keys,) @@ -113,21 +191,47 @@ def write_to_dataset( fs.mkdirs(prefix, exist_ok=True) filename = filename or uuid4().hex + ".parquet" full_path = fs.sep.join([prefix, filename]) - write_df = sub_df.copy(deep=False) - write_df.drop(columns=partition_cols, inplace=True) - with fs.open(full_path, mode="wb") as fil: - fil = ioutils.get_IOBase_writer(fil) - if return_metadata: - metadata.append( - write_df.to_parquet( - fil, - index=preserve_index, - metadata_file_path=fs.sep.join([subdir, filename]), - **kwargs, - ) - ) - else: - write_df.to_parquet(fil, index=preserve_index, **kwargs) + full_paths.append(full_path) + # write_df = sub_df.copy(deep=False) + # write_df.drop(columns=partition_cols, inplace=True) + + # with ExitStack() as stack: + # open_files = [ + # stack.enter_context(fs.open(path, mode="wb")) + # for path in full_paths + # ] + # file_objs = [ioutils.get_IOBase_writer(fil) for fil in open_files] + # write_df = df.drop(columns=partition_cols, inplace=False) + # if return_metadata: + # metadata.append( + # _write_parquet( + # write_df, + # file_objs, + # index=preserve_index, + # metadata_file_path=fs.sep.join([subdir, filename]), + # partitions_info=partitions_info, + # **kwargs, + # ) + # ) + # else: + # _write_parquet( + # write_df, + # file_objs, + # index=preserve_index, + # partitions_info=partitions_info, + # **kwargs, + # ) + # TODO: this is temp. we'll replace get_partitions with groupby + write_df = df.sort_values(partition_cols).drop( + columns=partition_cols, inplace=False + ) + _write_parquet( + write_df, + full_paths, + index=preserve_index, + partitions_info=partitions_info, + **kwargs, + ) else: filename = filename or uuid4().hex + ".parquet" @@ -669,6 +773,7 @@ def to_parquet( index=None, partition_cols=None, partition_file_name=None, + partition_offsets=None, statistics="ROWGROUP", metadata_file_path=None, int96_timestamps=False, @@ -680,17 +785,6 @@ def to_parquet( """{docstring}""" if engine == "cudf": - if partition_cols: - write_to_dataset( - df, - filename=partition_file_name, - partition_cols=partition_cols, - root_path=path, - preserve_index=index, - **kwargs, - ) - return - # Ensure that no columns dtype is 'category' for col in df.columns: if df[col].dtype.name == "category": @@ -699,39 +793,25 @@ def to_parquet( + "supported by the gpu accelerated parquet writer" ) - path_or_buf = ioutils.get_writer_filepath_or_buffer( - path, mode="wb", **kwargs - ) - if ioutils.is_fsspec_open_file(path_or_buf): - with path_or_buf as file_obj: - file_obj = ioutils.get_IOBase_writer(file_obj) - write_parquet_res = libparquet.write_parquet( - df, - path=file_obj, - index=index, - compression=compression, - statistics=statistics, - metadata_file_path=metadata_file_path, - int96_timestamps=int96_timestamps, - row_group_size_bytes=row_group_size_bytes, - row_group_size_rows=row_group_size_rows, - ) - else: - write_parquet_res = libparquet.write_parquet( + # TODO: propagate compression, statistics, int96, row_group_size_bytes, + # row_group_size_rows. + # TODO: warn against metadata_file_path being provided when partition + # cols specified + if partition_cols: + write_to_dataset( df, - path=path_or_buf, - index=index, - compression=compression, - statistics=statistics, - metadata_file_path=metadata_file_path, - int96_timestamps=int96_timestamps, - row_group_size_bytes=row_group_size_bytes, - row_group_size_rows=row_group_size_rows, + filename=partition_file_name, + partition_cols=partition_cols, + root_path=path, + preserve_index=index, + **kwargs, ) + return - return write_parquet_res + # TODO: re-enable non-partitioned case else: + # TODO: error if partition_offsets is specified without cudf engine # If index is empty set it to the expected default value of True if index is None: From 1e79453054cf3c6fdbd540c940a03e98467602fd Mon Sep 17 00:00:00 2001 From: Devavret Makkar Date: Mon, 13 Dec 2021 17:07:08 +0530 Subject: [PATCH 31/71] Review changes: reduce line size by aliasing the variable I keep using --- cpp/src/io/parquet/writer_impl.cu | 134 ++++++++++++++--------------- cpp/src/io/parquet/writer_impl.hpp | 2 +- 2 files changed, 64 insertions(+), 72 deletions(-) diff --git a/cpp/src/io/parquet/writer_impl.cu b/cpp/src/io/parquet/writer_impl.cu index b266a343791..c2875f37806 100644 --- a/cpp/src/io/parquet/writer_impl.cu +++ b/cpp/src/io/parquet/writer_impl.cu @@ -83,7 +83,6 @@ struct aggregate_metadata { aggregate_metadata(std::vector const& partitions, size_type num_columns, std::vector schema, - statistics_freq stats_granularity, std::vector> const& kv_md) : version(1), schema(std::move(schema)), files(partitions.size()) @@ -829,7 +828,7 @@ gpu::parquet_column_device_view parquet_column_view::get_device_view( void writer::impl::init_page_fragments(cudf::detail::hostdevice_2dvector& frag, device_span col_desc, - std::vector partitions, + host_span partitions, device_span part_frag_offset, uint32_t fragment_size) { @@ -1163,10 +1162,10 @@ void writer::impl::init_state() // Write file header file_header_s fhdr; fhdr.magic = parquet_magic; - for (size_t p = 0; p < out_sink_.size(); p++) { - out_sink_[p]->host_write(&fhdr, sizeof(fhdr)); - current_chunk_offset[p] = sizeof(file_header_s); + for (auto& sink : out_sink_) { + sink->host_write(&fhdr, sizeof(fhdr)); } + std::fill_n(current_chunk_offset.begin(), current_chunk_offset.size(), sizeof(file_header_s)); } void writer::impl::write(table_view const& table, std::vector const& partitions) @@ -1325,7 +1324,6 @@ void writer::impl::write(table_view const& table, std::vector co } } - // TODO: My gut says there should be a way to consolidate this with rg_to_part std::vector first_rg_in_part; std::exclusive_scan( num_rg_in_part.begin(), num_rg_in_part.end(), std::back_inserter(first_rg_in_part), 0); @@ -1339,47 +1337,46 @@ void writer::impl::write(table_view const& table, std::vector co size_type start_row = partitions[p].start_row; for (int r = 0; r < num_rg_in_part[p]; r++) { size_t global_r = global_rowgroup_base[p] + r; // Number of rowgroups already in file/part - uint32_t fragments_in_chunk = util::div_rounding_up_unsafe( - md->files[p].row_groups[global_r].num_rows, max_page_fragment_size); - md->files[p].row_groups[global_r].total_byte_size = 0; - md->files[p].row_groups[global_r].columns.resize(num_columns); + auto& row_group = md->files[p].row_groups[global_r]; + uint32_t fragments_in_chunk = + util::div_rounding_up_unsafe(row_group.num_rows, max_page_fragment_size); + row_group.total_byte_size = 0; + row_group.columns.resize(num_columns); for (int c = 0; c < num_columns; c++) { - gpu::EncColumnChunk* ck = &chunks[r + first_rg_in_part[p]][c]; + gpu::EncColumnChunk& ck = chunks[r + first_rg_in_part[p]][c]; - *ck = {}; - ck->col_desc = col_desc.device_ptr() + c; - ck->col_desc_id = c; - ck->fragments = &fragments.device_view()[c][f]; - ck->stats = + ck = {}; + ck.col_desc = col_desc.device_ptr() + c; + ck.col_desc_id = c; + ck.fragments = &fragments.device_view()[c][f]; + ck.stats = (not frag_stats.is_empty()) ? frag_stats.data() + c * num_fragments + f : nullptr; - ck->start_row = start_row; - ck->num_rows = (uint32_t)md->files[p].row_groups[global_r].num_rows; - ck->first_fragment = c * num_fragments + f; + ck.start_row = start_row; + ck.num_rows = (uint32_t)row_group.num_rows; + ck.first_fragment = c * num_fragments + f; auto chunk_fragments = fragments[c].subspan(f, fragments_in_chunk); // In fragment struct, add a pointer to the chunk it belongs to // In each fragment in chunk_fragments, update the chunk pointer here. for (auto& frag : chunk_fragments) { frag.chunk = &chunks.device_view()[r + first_rg_in_part[p]][c]; } - ck->num_values = std::accumulate( + ck.num_values = std::accumulate( chunk_fragments.begin(), chunk_fragments.end(), 0, [](uint32_t l, auto r) { return l + r.num_values; }); - ck->plain_data_size = std::accumulate( + ck.plain_data_size = std::accumulate( chunk_fragments.begin(), chunk_fragments.end(), 0, [](int sum, gpu::PageFragment frag) { return sum + frag.fragment_data_size; }); - md->files[p].row_groups[global_r].columns[c].meta_data.type = - parquet_columns[c].physical_type(); - md->files[p].row_groups[global_r].columns[c].meta_data.encodings = {Encoding::PLAIN, - Encoding::RLE}; - md->files[p].row_groups[global_r].columns[c].meta_data.path_in_schema = - parquet_columns[c].get_path_in_schema(); - md->files[p].row_groups[global_r].columns[c].meta_data.codec = UNCOMPRESSED; - md->files[p].row_groups[global_r].columns[c].meta_data.num_values = ck->num_values; + auto& column_chunk_meta = row_group.columns[c].meta_data; + column_chunk_meta.type = parquet_columns[c].physical_type(); + column_chunk_meta.encodings = {Encoding::PLAIN, Encoding::RLE}; + column_chunk_meta.path_in_schema = parquet_columns[c].get_path_in_schema(); + column_chunk_meta.codec = UNCOMPRESSED; + column_chunk_meta.num_values = ck.num_values; } f += fragments_in_chunk; - start_row += (uint32_t)md->files[p].row_groups[global_r].num_rows; + start_row += (uint32_t)row_group.num_rows; } } @@ -1486,11 +1483,11 @@ void writer::impl::write(table_view const& table, std::vector co auto bfr_c = static_cast(comp_bfr.data()); for (auto j = 0; j < batch_list[b]; j++, r++) { for (auto i = 0; i < num_columns; i++) { - gpu::EncColumnChunk* ck = &chunks[r][i]; - ck->uncompressed_bfr = bfr; - ck->compressed_bfr = bfr_c; - bfr += ck->bfr_size; - bfr_c += ck->compressed_size; + gpu::EncColumnChunk& ck = chunks[r][i]; + ck.uncompressed_bfr = bfr; + ck.compressed_bfr = bfr_c; + bfr += ck.bfr_size; + bfr_c += ck.compressed_size; } } } @@ -1531,32 +1528,32 @@ void writer::impl::write(table_view const& table, std::vector co : nullptr); std::vector> write_tasks; for (; r < rnext; r++) { - int p = rg_to_part[r]; - int global_r = global_rowgroup_base[p] + r - first_rg_in_part[p]; + int p = rg_to_part[r]; + int global_r = global_rowgroup_base[p] + r - first_rg_in_part[p]; + auto& row_group = md->files[p].row_groups[global_r]; for (auto i = 0; i < num_columns; i++) { - gpu::EncColumnChunk* ck = &chunks[r][i]; + gpu::EncColumnChunk& ck = chunks[r][i]; + auto& column_chunk_meta = row_group.columns[i].meta_data; uint8_t* dev_bfr; - if (ck->is_compressed) { - md->files[p].row_groups[global_r].columns[i].meta_data.codec = compression_; - dev_bfr = ck->compressed_bfr; + if (ck.is_compressed) { + column_chunk_meta.codec = compression_; + dev_bfr = ck.compressed_bfr; } else { - dev_bfr = ck->uncompressed_bfr; + dev_bfr = ck.uncompressed_bfr; } - if (out_sink_[p]->is_device_write_preferred(ck->compressed_size)) { + if (out_sink_[p]->is_device_write_preferred(ck.compressed_size)) { // let the writer do what it wants to retrieve the data from the gpu. write_tasks.push_back(out_sink_[p]->device_write_async( - dev_bfr + ck->ck_stat_size, ck->compressed_size, stream)); + dev_bfr + ck.ck_stat_size, ck.compressed_size, stream)); // we still need to do a (much smaller) memcpy for the statistics. - if (ck->ck_stat_size != 0) { - md->files[p].row_groups[global_r].columns[i].meta_data.statistics_blob.resize( - ck->ck_stat_size); - CUDA_TRY(cudaMemcpyAsync( - md->files[p].row_groups[global_r].columns[i].meta_data.statistics_blob.data(), - dev_bfr, - ck->ck_stat_size, - cudaMemcpyDeviceToHost, - stream.value())); + if (ck.ck_stat_size != 0) { + column_chunk_meta.statistics_blob.resize(ck.ck_stat_size); + CUDA_TRY(cudaMemcpyAsync(column_chunk_meta.statistics_blob.data(), + dev_bfr, + ck.ck_stat_size, + cudaMemcpyDeviceToHost, + stream.value())); stream.synchronize(); } } else { @@ -1571,29 +1568,24 @@ void writer::impl::write(table_view const& table, std::vector co // copy the full data CUDA_TRY(cudaMemcpyAsync(host_bfr.get(), dev_bfr, - ck->ck_stat_size + ck->compressed_size, + ck.ck_stat_size + ck.compressed_size, cudaMemcpyDeviceToHost, stream.value())); stream.synchronize(); - out_sink_[p]->host_write(host_bfr.get() + ck->ck_stat_size, ck->compressed_size); - if (ck->ck_stat_size != 0) { - md->files[p].row_groups[global_r].columns[i].meta_data.statistics_blob.resize( - ck->ck_stat_size); - memcpy(md->files[p].row_groups[global_r].columns[i].meta_data.statistics_blob.data(), - host_bfr.get(), - ck->ck_stat_size); + out_sink_[p]->host_write(host_bfr.get() + ck.ck_stat_size, ck.compressed_size); + if (ck.ck_stat_size != 0) { + column_chunk_meta.statistics_blob.resize(ck.ck_stat_size); + memcpy(column_chunk_meta.statistics_blob.data(), host_bfr.get(), ck.ck_stat_size); } } - md->files[p].row_groups[global_r].total_byte_size += ck->compressed_size; - md->files[p].row_groups[global_r].columns[i].meta_data.data_page_offset = - current_chunk_offset[p] + ((ck->use_dictionary) ? ck->dictionary_size : 0); - md->files[p].row_groups[global_r].columns[i].meta_data.dictionary_page_offset = - (ck->use_dictionary) ? current_chunk_offset[p] : 0; - md->files[p].row_groups[global_r].columns[i].meta_data.total_uncompressed_size = - ck->bfr_size; - md->files[p].row_groups[global_r].columns[i].meta_data.total_compressed_size = - ck->compressed_size; - current_chunk_offset[p] += ck->compressed_size; + row_group.total_byte_size += ck.compressed_size; + column_chunk_meta.data_page_offset = + current_chunk_offset[p] + ((ck.use_dictionary) ? ck.dictionary_size : 0); + column_chunk_meta.dictionary_page_offset = + (ck.use_dictionary) ? current_chunk_offset[p] : 0; + column_chunk_meta.total_uncompressed_size = ck.bfr_size; + column_chunk_meta.total_compressed_size = ck.compressed_size; + current_chunk_offset[p] += ck.compressed_size; } } for (auto const& task : write_tasks) { diff --git a/cpp/src/io/parquet/writer_impl.hpp b/cpp/src/io/parquet/writer_impl.hpp index 10537d72513..1cefb91c904 100644 --- a/cpp/src/io/parquet/writer_impl.hpp +++ b/cpp/src/io/parquet/writer_impl.hpp @@ -130,7 +130,7 @@ class writer::impl { */ void init_page_fragments(hostdevice_2dvector& frag, device_span col_desc, - std::vector partitions, + host_span partitions, device_span part_frag_offset, uint32_t fragment_size); From be83945fbffcd1a1441b5f52e1cc0df9811eb9ba Mon Sep 17 00:00:00 2001 From: Devavret Makkar Date: Mon, 13 Dec 2021 17:41:21 +0530 Subject: [PATCH 32/71] source/sink_info memeber privatisation --- cpp/include/cudf/io/parquet.hpp | 16 +++---- cpp/include/cudf/io/types.hpp | 64 ++++++++++++++------------ cpp/src/io/functions.cpp | 42 ++++++++--------- python/cudf/cudf/_lib/cpp/io/types.pxd | 10 ++-- 4 files changed, 67 insertions(+), 65 deletions(-) diff --git a/cpp/include/cudf/io/parquet.hpp b/cpp/include/cudf/io/parquet.hpp index 2f96c0aab47..f9c9d1b8f78 100644 --- a/cpp/include/cudf/io/parquet.hpp +++ b/cpp/include/cudf/io/parquet.hpp @@ -487,7 +487,7 @@ class parquet_writer_options { */ void set_partitions(std::vector partitions) { - CUDF_EXPECTS(partitions.size() == _sink.num_sinks, + CUDF_EXPECTS(partitions.size() == _sink.num_sinks(), "Mismatch between number of sinks and number of partitions"); _partitions = std::move(partitions); } @@ -506,7 +506,7 @@ class parquet_writer_options { */ void set_key_value_metadata(std::vector> metadata) { - CUDF_EXPECTS(metadata.size() == _sink.num_sinks, + CUDF_EXPECTS(metadata.size() == _sink.num_sinks(), "Mismatch between number of sinks and number of metadata maps"); _user_data = std::move(metadata); } @@ -541,7 +541,7 @@ class parquet_writer_options { */ void set_column_chunks_file_paths(std::vector file_paths) { - CUDF_EXPECTS(file_paths.size() == _sink.num_sinks, + CUDF_EXPECTS(file_paths.size() == _sink.num_sinks(), "Mismatch between number of sinks and number of chunk paths to set"); _column_chunks_file_paths = std::move(file_paths); } @@ -600,7 +600,7 @@ class parquet_writer_options_builder { */ parquet_writer_options_builder& partitions(std::vector partitions) { - CUDF_EXPECTS(partitions.size() == options._sink.num_sinks, + CUDF_EXPECTS(partitions.size() == options._sink.num_sinks(), "Mismatch between number of sinks and number of partitions"); options.set_partitions(std::move(partitions)); return *this; @@ -627,7 +627,7 @@ class parquet_writer_options_builder { parquet_writer_options_builder& key_value_metadata( std::vector> metadata) { - CUDF_EXPECTS(metadata.size() == options._sink.num_sinks, + CUDF_EXPECTS(metadata.size() == options._sink.num_sinks(), "Mismatch between number of sinks and number of metadata maps"); options._user_data = std::move(metadata); return *this; @@ -666,7 +666,7 @@ class parquet_writer_options_builder { */ parquet_writer_options_builder& column_chunks_file_paths(std::vector file_paths) { - CUDF_EXPECTS(file_paths.size() == options._sink.num_sinks, + CUDF_EXPECTS(file_paths.size() == options._sink.num_sinks(), "Mismatch between number of sinks and number of chunk paths to set"); options.set_column_chunks_file_paths(std::move(file_paths)); return *this; @@ -855,7 +855,7 @@ class chunked_parquet_writer_options { */ void set_key_value_metadata(std::vector> metadata) { - CUDF_EXPECTS(metadata.size() == _sink.num_sinks, + CUDF_EXPECTS(metadata.size() == _sink.num_sinks(), "Mismatch between number of sinks and number of metadata maps"); _user_data = std::move(metadata); } @@ -953,7 +953,7 @@ class chunked_parquet_writer_options_builder { chunked_parquet_writer_options_builder& key_value_metadata( std::vector> metadata) { - CUDF_EXPECTS(metadata.size() == options._sink.num_sinks, + CUDF_EXPECTS(metadata.size() == options._sink.num_sinks(), "Mismatch between number of sinks and number of metadata maps"); options.set_key_value_metadata(std::move(metadata)); return *this; diff --git a/cpp/include/cudf/io/types.hpp b/cpp/include/cudf/io/types.hpp index 9820eeaf977..512a90b3249 100644 --- a/cpp/include/cudf/io/types.hpp +++ b/cpp/include/cudf/io/types.hpp @@ -151,47 +151,48 @@ struct host_buffer { * @brief Source information for read interfaces */ struct source_info { - io_type type = io_type::FILEPATH; - std::vector> files; + std::vector> _files; source_info() = default; explicit source_info(std::vector const& file_paths) - : type(io_type::FILEPATH), filepaths(file_paths) + : _type(io_type::FILEPATH), _filepaths(file_paths) { } explicit source_info(std::string const& file_path) - : type(io_type::FILEPATH), filepaths({file_path}) + : _type(io_type::FILEPATH), _filepaths({file_path}) { } explicit source_info(std::vector const& host_buffers) - : type(io_type::HOST_BUFFER), buffers(host_buffers) + : _type(io_type::HOST_BUFFER), _buffers(host_buffers) { } explicit source_info(const char* host_data, size_t size) - : type(io_type::HOST_BUFFER), buffers({{host_data, size}}) + : _type(io_type::HOST_BUFFER), _buffers({{host_data, size}}) { } explicit source_info(std::vector const& sources) - : type(io_type::USER_IMPLEMENTED), user_sources(sources) + : _type(io_type::USER_IMPLEMENTED), _user_sources(sources) { } explicit source_info(cudf::io::datasource* source) - : type(io_type::USER_IMPLEMENTED), user_sources({source}) + : _type(io_type::USER_IMPLEMENTED), _user_sources({source}) { } - auto const& get_filepaths() const { return filepaths; } - auto const& get_buffers() const { return buffers; } - auto const& get_files() const { return files; } - auto const& get_user_sources() const { return user_sources; } + auto type() const { return _type; } + auto const& filepaths() const { return _filepaths; } + auto const& buffers() const { return _buffers; } + auto const& files() const { return _files; } + auto const& user_sources() const { return _user_sources; } private: - std::vector filepaths; - std::vector buffers; - std::vector user_sources; + io_type _type = io_type::FILEPATH; + std::vector _filepaths; + std::vector _buffers; + std::vector _user_sources; }; /** @@ -199,41 +200,44 @@ struct source_info { */ struct sink_info { sink_info() = default; - sink_info(size_t num_sinks) : type(io_type::VOID), num_sinks(num_sinks) {} + sink_info(size_t num_sinks) : _type(io_type::VOID), _num_sinks(num_sinks) {} explicit sink_info(std::vector const& file_paths) - : type(io_type::FILEPATH), num_sinks(file_paths.size()), filepaths(file_paths) + : _type(io_type::FILEPATH), _num_sinks(file_paths.size()), _filepaths(file_paths) { } - explicit sink_info(std::string const& file_path) : type(io_type::FILEPATH), filepaths({file_path}) + explicit sink_info(std::string const& file_path) + : _type(io_type::FILEPATH), _filepaths({file_path}) { } explicit sink_info(std::vector*> const& buffers) - : type(io_type::HOST_BUFFER), num_sinks(buffers.size()), buffers(buffers) + : _type(io_type::HOST_BUFFER), _num_sinks(buffers.size()), _buffers(buffers) { } - explicit sink_info(std::vector* buffer) : type(io_type::HOST_BUFFER), buffers({buffer}) {} + explicit sink_info(std::vector* buffer) : _type(io_type::HOST_BUFFER), _buffers({buffer}) {} explicit sink_info(std::vector const& user_sinks) - : type(io_type::USER_IMPLEMENTED), num_sinks(user_sinks.size()), user_sinks(user_sinks) + : _type(io_type::USER_IMPLEMENTED), _num_sinks(user_sinks.size()), _user_sinks(user_sinks) { } explicit sink_info(class cudf::io::data_sink* user_sink) - : type(io_type::USER_IMPLEMENTED), user_sinks({user_sink}) + : _type(io_type::USER_IMPLEMENTED), _user_sinks({user_sink}) { } - auto const& get_filepaths() const { return filepaths; } - auto const& get_buffers() const { return buffers; } - auto const& get_user_sinks() const { return user_sinks; } - io_type type = io_type::VOID; - size_t num_sinks = 1; + auto type() const { return _type; } + auto num_sinks() const { return _num_sinks; } + auto const& filepaths() const { return _filepaths; } + auto const& buffers() const { return _buffers; } + auto const& user_sinks() const { return _user_sinks; } private: - std::vector filepaths; - std::vector*> buffers; - std::vector user_sinks; + io_type _type = io_type::VOID; + size_t _num_sinks = 1; + std::vector _filepaths; + std::vector*> _buffers; + std::vector _user_sinks; }; class table_input_metadata; diff --git a/cpp/src/io/functions.cpp b/cpp/src/io/functions.cpp index 0944e6e3008..df7d6795ee3 100644 --- a/cpp/src/io/functions.cpp +++ b/cpp/src/io/functions.cpp @@ -111,33 +111,33 @@ std::vector> make_datasources(source_info size_t range_offset = 0, size_t range_size = 0) { - switch (info.type) { + switch (info.type()) { case io_type::FILEPATH: { auto sources = std::vector>(); - for (auto const& filepath : info.get_filepaths()) { + for (auto const& filepath : info.filepaths()) { sources.emplace_back(cudf::io::datasource::create(filepath, range_offset, range_size)); } return sources; } - case io_type::HOST_BUFFER: return cudf::io::datasource::create(info.get_buffers()); - case io_type::USER_IMPLEMENTED: return cudf::io::datasource::create(info.get_user_sources()); + case io_type::HOST_BUFFER: return cudf::io::datasource::create(info.buffers()); + case io_type::USER_IMPLEMENTED: return cudf::io::datasource::create(info.user_sources()); default: CUDF_FAIL("Unsupported source type"); } } std::vector> make_datasinks(sink_info const& info) { - switch (info.type) { - case io_type::FILEPATH: return cudf::io::data_sink::create(info.get_filepaths()); - case io_type::HOST_BUFFER: return cudf::io::data_sink::create(info.get_buffers()); + switch (info.type()) { + case io_type::FILEPATH: return cudf::io::data_sink::create(info.filepaths()); + case io_type::HOST_BUFFER: return cudf::io::data_sink::create(info.buffers()); case io_type::VOID: { std::vector> sinks; - for (size_t i = 0; i < info.num_sinks; ++i) { + for (size_t i = 0; i < info.num_sinks(); ++i) { sinks.push_back(cudf::io::data_sink::create()); } return sinks; } - case io_type::USER_IMPLEMENTED: return cudf::io::data_sink::create(info.get_user_sinks()); + case io_type::USER_IMPLEMENTED: return cudf::io::data_sink::create(info.user_sinks()); default: CUDF_FAIL("Unsupported sink type"); } } @@ -162,9 +162,9 @@ compression_type infer_compression_type(compression_type compression, source_inf { if (compression != compression_type::AUTO) { return compression; } - if (info.type != io_type::FILEPATH) { return compression_type::NONE; } + if (info.type() != io_type::FILEPATH) { return compression_type::NONE; } - auto filepath = info.get_filepaths()[0]; + auto filepath = info.filepaths()[0]; // Attempt to infer from the file extension const auto pos = filepath.find_last_of('.'); @@ -242,18 +242,16 @@ raw_orc_statistics read_raw_orc_statistics(source_info const& src_info) { // Get source to read statistics from std::unique_ptr source; - if (src_info.type == io_type::FILEPATH) { - CUDF_EXPECTS(src_info.get_filepaths().size() == 1, + if (src_info.type() == io_type::FILEPATH) { + CUDF_EXPECTS(src_info.filepaths().size() == 1, "Only a single source is currently supported."); + source = cudf::io::datasource::create(src_info.filepaths()[0]); + } else if (src_info.type() == io_type::HOST_BUFFER) { + CUDF_EXPECTS(src_info.buffers().size() == 1, "Only a single source is currently supported."); + source = cudf::io::datasource::create(src_info.buffers()[0]); + } else if (src_info.type() == io_type::USER_IMPLEMENTED) { + CUDF_EXPECTS(src_info.user_sources().size() == 1, "Only a single source is currently supported."); - source = cudf::io::datasource::create(src_info.get_filepaths()[0]); - } else if (src_info.type == io_type::HOST_BUFFER) { - CUDF_EXPECTS(src_info.get_buffers().size() == 1, - "Only a single source is currently supported."); - source = cudf::io::datasource::create(src_info.get_buffers()[0]); - } else if (src_info.type == io_type::USER_IMPLEMENTED) { - CUDF_EXPECTS(src_info.get_user_sources().size() == 1, - "Only a single source is currently supported."); - source = cudf::io::datasource::create(src_info.get_user_sources()[0]); + source = cudf::io::datasource::create(src_info.user_sources()[0]); } else { CUDF_FAIL("Unsupported source type"); } diff --git a/python/cudf/cudf/_lib/cpp/io/types.pxd b/python/cudf/cudf/_lib/cpp/io/types.pxd index 1217f09f99c..40a056b46e0 100644 --- a/python/cudf/cudf/_lib/cpp/io/types.pxd +++ b/python/cudf/cudf/_lib/cpp/io/types.pxd @@ -82,8 +82,8 @@ cdef extern from "cudf/io/types.hpp" \ cdef cppclass source_info: io_type type - const vector[string]& get_filepaths() except + - const vector[host_buffer]& get_buffers() except + + const vector[string]& filepaths() except + + const vector[host_buffer]& buffers() except + vector[shared_ptr[CRandomAccessFile]] files source_info() except + @@ -93,9 +93,9 @@ cdef extern from "cudf/io/types.hpp" \ cdef cppclass sink_info: io_type type - const vector[string]& get_filepaths() - const vector[vector[char] *]& get_buffers() - const vector[data_sink *]& get_user_sinks() + const vector[string]& filepaths() + const vector[vector[char] *]& buffers() + const vector[data_sink *]& user_sinks() sink_info() except + sink_info(string file_path) except + From e53731423afdf9c75f57d07d649c88fee83c030f Mon Sep 17 00:00:00 2001 From: Devavret Makkar Date: Mon, 13 Dec 2021 17:41:43 +0530 Subject: [PATCH 33/71] aggregate metadata privatisation --- cpp/src/io/parquet/writer_impl.cu | 32 ++++++++++++++++++++----------- 1 file changed, 21 insertions(+), 11 deletions(-) diff --git a/cpp/src/io/parquet/writer_impl.cu b/cpp/src/io/parquet/writer_impl.cu index c2875f37806..57901fc7351 100644 --- a/cpp/src/io/parquet/writer_impl.cu +++ b/cpp/src/io/parquet/writer_impl.cu @@ -156,6 +156,20 @@ struct aggregate_metadata { return merged_md; } + std::vector num_row_groups_per_file() + { + std::vector global_rowgroup_base; + std::transform(this->files.begin(), + this->files.end(), + std::back_inserter(global_rowgroup_base), + [](auto const& part) { return part.row_groups.size(); }); + return global_rowgroup_base; + } + + auto& file(size_t p) { return files[p]; } + size_t num_files() const { return files.size(); } + + private: int32_t version = 0; std::vector schema; struct per_file_metadata { @@ -1266,11 +1280,7 @@ void writer::impl::write(table_view const& table, std::vector co fragments, col_desc, partitions, d_part_frag_offset, max_page_fragment_size); } - std::vector global_rowgroup_base; - std::transform(md->files.begin(), - md->files.end(), - std::back_inserter(global_rowgroup_base), - [](auto const& part) { return part.row_groups.size(); }); + std::vector const global_rowgroup_base = md->num_row_groups_per_file(); // Decide row group boundaries based on uncompressed data size int num_rowgroups = 0; @@ -1292,7 +1302,7 @@ void writer::impl::write(table_view const& table, std::vector co if (f > first_frag_in_rg && // There has to be at least one fragment in row group (curr_rg_data_size + fragment_data_size > max_row_group_size || curr_rg_num_rows + fragment_num_rows > max_row_group_rows)) { - auto& rg = md->files[p].row_groups.emplace_back(); + auto& rg = md->file(p).row_groups.emplace_back(); rg.num_rows = curr_rg_num_rows; num_rowgroups++; num_rg_in_part[p]++; @@ -1305,7 +1315,7 @@ void writer::impl::write(table_view const& table, std::vector co // TODO: (wishful) refactor to consolidate with above if block if (f == last_frag_in_part) { - auto& rg = md->files[p].row_groups.emplace_back(); + auto& rg = md->file(p).row_groups.emplace_back(); rg.num_rows = curr_rg_num_rows; num_rowgroups++; num_rg_in_part[p]++; @@ -1337,7 +1347,7 @@ void writer::impl::write(table_view const& table, std::vector co size_type start_row = partitions[p].start_row; for (int r = 0; r < num_rg_in_part[p]; r++) { size_t global_r = global_rowgroup_base[p] + r; // Number of rowgroups already in file/part - auto& row_group = md->files[p].row_groups[global_r]; + auto& row_group = md->file(p).row_groups[global_r]; uint32_t fragments_in_chunk = util::div_rounding_up_unsafe(row_group.num_rows, max_page_fragment_size); row_group.total_byte_size = 0; @@ -1387,7 +1397,7 @@ void writer::impl::write(table_view const& table, std::vector co size_t global_rg = global_rowgroup_base[p] + rg; for (int col = 0; col < num_columns; col++) { if (chunks.host_view()[rg][col].use_dictionary) { - md->files[p].row_groups[global_rg].columns[col].meta_data.encodings.push_back( + md->file(p).row_groups[global_rg].columns[col].meta_data.encodings.push_back( Encoding::PLAIN_DICTIONARY); } } @@ -1530,7 +1540,7 @@ void writer::impl::write(table_view const& table, std::vector co for (; r < rnext; r++) { int p = rg_to_part[r]; int global_r = global_rowgroup_base[p] + r - first_rg_in_part[p]; - auto& row_group = md->files[p].row_groups[global_r]; + auto& row_group = md->file(p).row_groups[global_r]; for (auto i = 0; i < num_columns; i++) { gpu::EncColumnChunk& ck = chunks[r][i]; auto& column_chunk_meta = row_group.columns[i].meta_data; @@ -1615,7 +1625,7 @@ std::unique_ptr> writer::impl::close( // Optionally output raw file metadata with the specified column chunk file path if (column_chunks_file_path.size() > 0) { - CUDF_EXPECTS(column_chunks_file_path.size() == md->files.size(), + CUDF_EXPECTS(column_chunks_file_path.size() == md->num_files(), "Expected one column chunk path per output file"); md->set_file_paths(column_chunks_file_path); file_header_s fhdr = {parquet_magic}; From c53fea75fc0837d876b07fbbc8cac33d9b4a24a7 Mon Sep 17 00:00:00 2001 From: Devavret Makkar Date: Mon, 13 Dec 2021 23:15:52 +0530 Subject: [PATCH 34/71] Fix a private member access --- cpp/src/io/parquet/writer_impl.cu | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/cpp/src/io/parquet/writer_impl.cu b/cpp/src/io/parquet/writer_impl.cu index 8eaabf8b4df..aceb3bfbec1 100644 --- a/cpp/src/io/parquet/writer_impl.cu +++ b/cpp/src/io/parquet/writer_impl.cu @@ -166,6 +166,10 @@ struct aggregate_metadata { return global_rowgroup_base; } + bool schema_matches(std::vector const& schema) const + { + return this->schema == schema; + } auto& file(size_t p) { return files[p]; } size_t num_files() const { return files.size(); } @@ -1226,7 +1230,7 @@ void writer::impl::write(table_view const& table, std::vector co partitions, num_columns, std::move(this_table_schema), stats_granularity_, kv_md); } else { // verify the user isn't passing mismatched tables - CUDF_EXPECTS(md->schema == this_table_schema, + CUDF_EXPECTS(md->schema_matches(this_table_schema), "Mismatch in schema between multiple calls to write_chunk"); md->update_files(partitions); From 085442c701cf2c90ad71743bf650082e0d0c9cdf Mon Sep 17 00:00:00 2001 From: Devavret Makkar Date: Tue, 14 Dec 2021 04:58:33 +0530 Subject: [PATCH 35/71] Working for IOBase --- python/cudf/cudf/_lib/cpp/io/types.pxd | 1 + python/cudf/cudf/_lib/io/utils.pyx | 21 ++++---- python/cudf/cudf/io/parquet.py | 70 +++++++++++--------------- python/cudf/cudf/tests/test_parquet.py | 5 ++ 4 files changed, 47 insertions(+), 50 deletions(-) diff --git a/python/cudf/cudf/_lib/cpp/io/types.pxd b/python/cudf/cudf/_lib/cpp/io/types.pxd index e4387397474..5e58134809d 100644 --- a/python/cudf/cudf/_lib/cpp/io/types.pxd +++ b/python/cudf/cudf/_lib/cpp/io/types.pxd @@ -109,6 +109,7 @@ cdef extern from "cudf/io/types.hpp" \ sink_info(vector[string] file_path) except + sink_info(vector[char] * buffer) except + sink_info(data_sink * user_sink) except + + sink_info(vector[data_sink *] user_sink) except + cdef extern from "cudf/io/data_sink.hpp" \ diff --git a/python/cudf/cudf/_lib/io/utils.pyx b/python/cudf/cudf/_lib/io/utils.pyx index 4a902165bac..f490e0dbc5e 100644 --- a/python/cudf/cudf/_lib/io/utils.pyx +++ b/python/cudf/cudf/_lib/io/utils.pyx @@ -80,7 +80,8 @@ cdef source_info make_source_info(list src) except*: # Converts the Python sink input to libcudf++ IO sink_info. cdef sink_info make_sinks_info(list src, vector[unique_ptr[data_sink]] & sink) except*: - cdef vector[data_sink] data_sinks + cdef vector[data_sink *] data_sinks + cdef vector[string] paths # TODO: re-enable after paths splits is known to work # if isinstance(src[0], io.StringIO): # for s in src: @@ -95,15 +96,15 @@ cdef sink_info make_sinks_info(list src, vector[unique_ptr[data_sink]] & sink) e # raise NotImplementedError(f"Unsupported encoding {src.encoding}") # sink.reset(new iobase_data_sink(src.buffer)) # return sink_info(sink.get()) - # if isinstance(src[0], io.IOBase): - # for s in src: - # data_sinks.push_back(new iobase_data_sink(s)) - # return sink_info(sink.get()) - # elif isinstance(src[0], (basestring, os.PathLike)): - cdef vector[string] paths - for s in src: - paths.push_back( os.path.expanduser(s).encode()) - return sink_info(move(paths)) + if isinstance(src[0], io.IOBase): + for s in src: + sink.push_back(unique_ptr[data_sink](new iobase_data_sink(s))) + data_sinks.push_back(sink.back().get()) + return sink_info(data_sinks) + elif isinstance(src[0], (basestring, os.PathLike)): + for s in src: + paths.push_back( os.path.expanduser(s).encode()) + return sink_info(move(paths)) # else: # raise TypeError("Unrecognized input type: {}".format(type(src))) diff --git a/python/cudf/cudf/io/parquet.py b/python/cudf/cudf/io/parquet.py index 62461e3fd37..ea71d880505 100644 --- a/python/cudf/cudf/io/parquet.py +++ b/python/cudf/cudf/io/parquet.py @@ -192,46 +192,36 @@ def write_to_dataset( filename = filename or uuid4().hex + ".parquet" full_path = fs.sep.join([prefix, filename]) full_paths.append(full_path) - # write_df = sub_df.copy(deep=False) - # write_df.drop(columns=partition_cols, inplace=True) - - # with ExitStack() as stack: - # open_files = [ - # stack.enter_context(fs.open(path, mode="wb")) - # for path in full_paths - # ] - # file_objs = [ioutils.get_IOBase_writer(fil) for fil in open_files] - # write_df = df.drop(columns=partition_cols, inplace=False) - # if return_metadata: - # metadata.append( - # _write_parquet( - # write_df, - # file_objs, - # index=preserve_index, - # metadata_file_path=fs.sep.join([subdir, filename]), - # partitions_info=partitions_info, - # **kwargs, - # ) - # ) - # else: - # _write_parquet( - # write_df, - # file_objs, - # index=preserve_index, - # partitions_info=partitions_info, - # **kwargs, - # ) - # TODO: this is temp. we'll replace get_partitions with groupby - write_df = df.sort_values(partition_cols).drop( - columns=partition_cols, inplace=False - ) - _write_parquet( - write_df, - full_paths, - index=preserve_index, - partitions_info=partitions_info, - **kwargs, - ) + + with ExitStack() as stack: + open_files = [ + stack.enter_context(fs.open(path, mode="wb")) + for path in full_paths + ] + file_objs = [ioutils.get_IOBase_writer(fil) for fil in open_files] + # TODO: this is temp. we'll replace get_partitions with groupby + write_df = df.sort_values(partition_cols).drop( + columns=partition_cols, inplace=False + ) + if return_metadata: + metadata.append( + _write_parquet( + write_df, + file_objs, + index=preserve_index, + metadata_file_path=fs.sep.join([subdir, filename]), + partitions_info=partitions_info, + **kwargs, + ) + ) + else: + _write_parquet( + write_df, + file_objs, + index=preserve_index, + partitions_info=partitions_info, + **kwargs, + ) else: filename = filename or uuid4().hex + ".parquet" diff --git a/python/cudf/cudf/tests/test_parquet.py b/python/cudf/cudf/tests/test_parquet.py index c52dab5c72f..2d7bebf30b8 100644 --- a/python/cudf/cudf/tests/test_parquet.py +++ b/python/cudf/cudf/tests/test_parquet.py @@ -2294,3 +2294,8 @@ def test_parquet_writer_row_group_size( math.ceil(num_rows / size_rows), math.ceil(8 * num_rows / size_bytes) ) assert expected_num_rows == row_groups + + +def test_partitioned(): + df = cudf.DataFrame({"a": [1, 1, 2, 2, 1], "b": [9, 8, 7, 6, 5]}) + cudf.io.parquet.write_to_dataset(df, "./", partition_cols="a") From 099093dbaa0eafe75a184d0dfec1e6053c7f1315 Mon Sep 17 00:00:00 2001 From: Devavret Makkar Date: Tue, 14 Dec 2021 05:25:49 +0530 Subject: [PATCH 36/71] Started to use groupby instead of special partition logic --- python/cudf/cudf/io/parquet.py | 19 ++++++++----------- 1 file changed, 8 insertions(+), 11 deletions(-) diff --git a/python/cudf/cudf/io/parquet.py b/python/cudf/cudf/io/parquet.py index ea71d880505..52c0b280899 100644 --- a/python/cudf/cudf/io/parquet.py +++ b/python/cudf/cudf/io/parquet.py @@ -164,12 +164,13 @@ def write_to_dataset( partition_offsets, partition_dfs = _get_partition_groups( df, partition_cols, preserve_index=preserve_index ) + part_names, part_offsets, _, grouped_df = df.groupby( + partition_cols + )._grouped() + grouped_df.drop(columns=partition_cols, inplace=True) partitions_info = [ - ( - partition_offsets[i], - partition_offsets[i + 1] - partition_offsets[i], - ) - for i in range(0, len(partition_offsets) - 1) + (part_offsets[i], part_offsets[i + 1] - part_offsets[i]) + for i in range(0, len(part_offsets) - 1) ] full_paths = [] @@ -199,14 +200,10 @@ def write_to_dataset( for path in full_paths ] file_objs = [ioutils.get_IOBase_writer(fil) for fil in open_files] - # TODO: this is temp. we'll replace get_partitions with groupby - write_df = df.sort_values(partition_cols).drop( - columns=partition_cols, inplace=False - ) if return_metadata: metadata.append( _write_parquet( - write_df, + grouped_df, file_objs, index=preserve_index, metadata_file_path=fs.sep.join([subdir, filename]), @@ -216,7 +213,7 @@ def write_to_dataset( ) else: _write_parquet( - write_df, + grouped_df, file_objs, index=preserve_index, partitions_info=partitions_info, From 2ffec7efd1b0e7786f7568d647a0ccf8d32461ef Mon Sep 17 00:00:00 2001 From: Devavret Makkar Date: Tue, 14 Dec 2021 17:26:17 +0530 Subject: [PATCH 37/71] Avoid multiple d->h iloc using groupby keys --- python/cudf/cudf/io/parquet.py | 16 ++++------------ 1 file changed, 4 insertions(+), 12 deletions(-) diff --git a/python/cudf/cudf/io/parquet.py b/python/cudf/cudf/io/parquet.py index 52c0b280899..f68aa4143ee 100644 --- a/python/cudf/cudf/io/parquet.py +++ b/python/cudf/cudf/io/parquet.py @@ -172,21 +172,13 @@ def write_to_dataset( (part_offsets[i], part_offsets[i + 1] - part_offsets[i]) for i in range(0, len(part_offsets) - 1) ] + # Copy the entire keys df in one operation rather than using iloc + part_names = part_names.to_pandas().to_frame(index=False) full_paths = [] - for _, sub_df in enumerate(partition_dfs): - if sub_df is None or len(sub_df) == 0: - continue - # TODO: This will cause multiple device reads. When we use groupby, - # we get a keys column which we can transfer in a single d->h copy - keys = tuple([sub_df[col].iloc[0] for col in partition_cols]) - if not isinstance(keys, tuple): - keys = (keys,) + for keys in part_names.itertuples(index=False): subdir = fs.sep.join( - [ - "{colname}={value}".format(colname=name, value=val) - for name, val in zip(partition_cols, keys) - ] + [f"{name}={val}" for name, val in zip(partition_cols, keys)] ) prefix = fs.sep.join([root_path, subdir]) fs.mkdirs(prefix, exist_ok=True) From 5dae74dee9b721b36393cdc4911c19c4e8f9a648 Mon Sep 17 00:00:00 2001 From: Devavret Makkar Date: Wed, 15 Dec 2021 05:43:19 +0530 Subject: [PATCH 38/71] Index fixes - preserve_index now works - RangeIndex now preserved when using partitions - bye bye _get_partition_groups --- python/cudf/cudf/_lib/parquet.pyx | 17 +++++++++++++---- python/cudf/cudf/io/parquet.py | 24 ------------------------ 2 files changed, 13 insertions(+), 28 deletions(-) diff --git a/python/cudf/cudf/_lib/parquet.pyx b/python/cudf/cudf/_lib/parquet.pyx index 380165b9188..43e240155f5 100644 --- a/python/cudf/cudf/_lib/parquet.pyx +++ b/python/cudf/cudf/_lib/parquet.pyx @@ -329,9 +329,18 @@ cpdef write_parquet( table[name]._column, tbl_meta.get().column_metadata[i] ) - pandas_metadata = generate_pandas_metadata(table, index) - user_data.resize(1) - user_data.back()[str.encode("pandas")] = str.encode(pandas_metadata) + cdef map[string, string] tmp_user_data + if partitions_info is not None: + for start_row, num_row in partitions_info: + partitioned_df = table.iloc[start_row: start_row+ num_row].copy(deep=False) + pandas_metadata = generate_pandas_metadata(partitioned_df, index) + tmp_user_data[str.encode("pandas")] = str.encode(pandas_metadata) + user_data.push_back(tmp_user_data) + tmp_user_data.clear() + else: + pandas_metadata = generate_pandas_metadata(table, index) + tmp_user_data[str.encode("pandas")] = str.encode(pandas_metadata) + user_data.push_back(tmp_user_data) cdef cudf_io_types.compression_type comp_type = _get_comp_type(compression) cdef cudf_io_types.statistics_freq stat_freq = _get_stat_freq(statistics) @@ -349,7 +358,7 @@ cpdef write_parquet( parquet_writer_options.builder(sink, tv) .partitions(partitions) # move if possible .metadata(tbl_meta.get()) - # .key_value_metadata(move(user_data)) + .key_value_metadata(move(user_data)) .compression(comp_type) .stats_level(stat_freq) .int96_timestamps(_int96_timestamps) diff --git a/python/cudf/cudf/io/parquet.py b/python/cudf/cudf/io/parquet.py index f68aa4143ee..c7e8d71bfcb 100644 --- a/python/cudf/cudf/io/parquet.py +++ b/python/cudf/cudf/io/parquet.py @@ -82,26 +82,6 @@ def _write_parquet( return write_parquet_res -def _get_partition_groups(df, partition_cols, preserve_index=False): - # TODO: We can use groupby functionality here after cudf#4346. - # Longer term, we want more slicing logic to be pushed down - # into cpp. For example, it would be best to pass libcudf - # a single sorted table with group offsets). - df = df.sort_values(partition_cols) - if not preserve_index: - df = df.reset_index(drop=True) - divisions = df[partition_cols].drop_duplicates(ignore_index=True) - splits = df[partition_cols].searchsorted(divisions, side="left") - splits = splits.tolist() + [len(df[partition_cols])] - return ( - splits, - [ - df.iloc[splits[i] : splits[i + 1]].copy(deep=False) - for i in range(0, len(splits) - 1) - ], - ) - - # Logic chosen to match: https://arrow.apache.org/ # docs/_modules/pyarrow/parquet.html#write_to_dataset def write_to_dataset( @@ -160,10 +140,6 @@ def write_to_dataset( if len(data_cols) == 0: raise ValueError("No data left to save outside partition columns") - # Loop through the partition groups - partition_offsets, partition_dfs = _get_partition_groups( - df, partition_cols, preserve_index=preserve_index - ) part_names, part_offsets, _, grouped_df = df.groupby( partition_cols )._grouped() From 120f32efb003b84c991959e0fed5ee685565c968 Mon Sep 17 00:00:00 2001 From: Devavret Makkar Date: Wed, 15 Dec 2021 17:28:17 +0530 Subject: [PATCH 39/71] Re-enabling non-partitioned case --- python/cudf/cudf/_lib/cpp/io/parquet.pxd | 3 +++ python/cudf/cudf/_lib/parquet.pyx | 8 ++++---- python/cudf/cudf/io/parquet.py | 23 ++++++++++++++++++++--- 3 files changed, 27 insertions(+), 7 deletions(-) diff --git a/python/cudf/cudf/_lib/cpp/io/parquet.pxd b/python/cudf/cudf/_lib/cpp/io/parquet.pxd index 338ac0d6f88..07b312361f2 100644 --- a/python/cudf/cudf/_lib/cpp/io/parquet.pxd +++ b/python/cudf/cudf/_lib/cpp/io/parquet.pxd @@ -77,6 +77,9 @@ cdef extern from "cudf/io/parquet.hpp" namespace "cudf::io" nogil: size_t get_row_group_size_bytes() except+ size_type get_row_group_size_rows() except+ + void set_partitions( + vector[cudf_io_types.partition_info] partitions + ) except + void set_metadata( cudf_io_types.table_input_metadata *m ) except + diff --git a/python/cudf/cudf/_lib/parquet.pyx b/python/cudf/cudf/_lib/parquet.pyx index 43e240155f5..f3f385a8e24 100644 --- a/python/cudf/cudf/_lib/parquet.pyx +++ b/python/cudf/cudf/_lib/parquet.pyx @@ -349,14 +349,10 @@ cpdef write_parquet( cdef vector[string] c_column_chunks_file_paths cdef bool _int96_timestamps = int96_timestamps cdef vector[cudf_io_types.partition_info] partitions - if partitions_info is not None: - for part in partitions_info: - partitions.push_back(cudf_io_types.partition_info(part[0], part[1])) # Perform write cdef parquet_writer_options args = move( parquet_writer_options.builder(sink, tv) - .partitions(partitions) # move if possible .metadata(tbl_meta.get()) .key_value_metadata(move(user_data)) .compression(comp_type) @@ -364,6 +360,10 @@ cpdef write_parquet( .int96_timestamps(_int96_timestamps) .build() ) + if partitions_info is not None: + for part in partitions_info: + partitions.push_back(cudf_io_types.partition_info(part[0], part[1])) + args.set_partitions(move(partitions)) if metadata_file_path is not None: c_column_chunks_file_paths.push_back(str.encode(metadata_file_path)) args.set_column_chunks_file_paths(move(c_column_chunks_file_paths)) diff --git a/python/cudf/cudf/io/parquet.py b/python/cudf/cudf/io/parquet.py index c7e8d71bfcb..59701f2dccb 100644 --- a/python/cudf/cudf/io/parquet.py +++ b/python/cudf/cudf/io/parquet.py @@ -37,9 +37,14 @@ def _write_parquet( # If partitions_info are specified then paths should be list like. # In either case, both should be of same length. # Partition offsets should be a list of 2-tuples - if is_list_like(paths) or partitions_info is not None: - if not len(paths) == len(partitions_info): - ValueError("") + # if is_list_like(paths) and len(paths) > 1: + # if partitions_info is None: + # ValueError("partition info is required for multiple paths") + # elif not is_list_like(partitions_info): + # ValueError("partition info must be list-like") + + # if not len(paths) == len(partitions_info): + # ValueError("") paths_or_bufs = [ ioutils.get_writer_filepath_or_buffer(path, mode="wb", **kwargs) @@ -764,6 +769,18 @@ def to_parquet( return # TODO: re-enable non-partitioned case + _write_parquet( + df, + paths=[path], + compression=compression, + index=index, + statistics=statistics, + metadata_file_path=metadata_file_path, + int96_timestamps=int96_timestamps, + row_group_size_bytes=row_group_size_bytes, + row_group_size_rows=row_group_size_rows, + **kwargs, + ) else: # TODO: error if partition_offsets is specified without cudf engine From 39ec9a0460fe66a97a45a62ee526490ca2ba16e9 Mon Sep 17 00:00:00 2001 From: Devavret Makkar Date: Thu, 16 Dec 2021 01:03:17 +0530 Subject: [PATCH 40/71] remember to return metadata. remove temp test --- python/cudf/cudf/io/parquet.py | 2 +- python/cudf/cudf/tests/test_parquet.py | 5 ----- 2 files changed, 1 insertion(+), 6 deletions(-) diff --git a/python/cudf/cudf/io/parquet.py b/python/cudf/cudf/io/parquet.py index 59701f2dccb..fb1da66a524 100644 --- a/python/cudf/cudf/io/parquet.py +++ b/python/cudf/cudf/io/parquet.py @@ -769,7 +769,7 @@ def to_parquet( return # TODO: re-enable non-partitioned case - _write_parquet( + return _write_parquet( df, paths=[path], compression=compression, diff --git a/python/cudf/cudf/tests/test_parquet.py b/python/cudf/cudf/tests/test_parquet.py index 2d7bebf30b8..c52dab5c72f 100644 --- a/python/cudf/cudf/tests/test_parquet.py +++ b/python/cudf/cudf/tests/test_parquet.py @@ -2294,8 +2294,3 @@ def test_parquet_writer_row_group_size( math.ceil(num_rows / size_rows), math.ceil(8 * num_rows / size_bytes) ) assert expected_num_rows == row_groups - - -def test_partitioned(): - df = cudf.DataFrame({"a": [1, 1, 2, 2, 1], "b": [9, 8, 7, 6, 5]}) - cudf.io.parquet.write_to_dataset(df, "./", partition_cols="a") From 43d35cb915b0d61242b001447d348e714cb5f375 Mon Sep 17 00:00:00 2001 From: Devavret Makkar Date: Thu, 16 Dec 2021 02:10:49 +0530 Subject: [PATCH 41/71] actually drop index when asked to. now mimics the pre-refactor behaviour --- python/cudf/cudf/io/parquet.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/python/cudf/cudf/io/parquet.py b/python/cudf/cudf/io/parquet.py index fb1da66a524..366fc39cc37 100644 --- a/python/cudf/cudf/io/parquet.py +++ b/python/cudf/cudf/io/parquet.py @@ -148,6 +148,8 @@ def write_to_dataset( part_names, part_offsets, _, grouped_df = df.groupby( partition_cols )._grouped() + if not preserve_index: + grouped_df.reset_index(drop=True, inplace=True) grouped_df.drop(columns=partition_cols, inplace=True) partitions_info = [ (part_offsets[i], part_offsets[i + 1] - part_offsets[i]) From 7d7444a4afa3fbe9c341af3f16e0fb93327f04b6 Mon Sep 17 00:00:00 2001 From: Devavret Makkar Date: Fri, 17 Dec 2021 03:23:55 +0530 Subject: [PATCH 42/71] Fix flaky test --- python/cudf/cudf/tests/test_parquet.py | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/python/cudf/cudf/tests/test_parquet.py b/python/cudf/cudf/tests/test_parquet.py index c52dab5c72f..9a66de8a3a6 100644 --- a/python/cudf/cudf/tests/test_parquet.py +++ b/python/cudf/cudf/tests/test_parquet.py @@ -1706,21 +1706,19 @@ def test_read_parquet_partitioned_filtered( row_groups=row_groups, categorical_partitions=use_cat, ) + expect["b"] = expect["b"].astype(str) + expect["c"] = expect["c"].astype(int) if use_cat: assert got.dtypes["b"] == "category" assert got.dtypes["c"] == "category" + got["b"] = got["b"].astype(str) + got["c"] = got["c"].astype(int) else: # Check that we didn't get categorical # columns, but convert back to categorical # for comparison with pandas assert got.dtypes["b"] == "object" assert got.dtypes["c"] == "int" - got["b"] = pd.Categorical( - got["b"].to_pandas(), categories=list("abcd") - ) - got["c"] = pd.Categorical( - got["c"].to_pandas(), categories=np.arange(4) - ) assert_eq(expect, got) # Filter on non-partitioned column. From e66d8550c8560e41e18bfcfea58ac502ff49cda7 Mon Sep 17 00:00:00 2001 From: Devavret Makkar Date: Fri, 17 Dec 2021 17:43:51 +0530 Subject: [PATCH 43/71] Metadata file paths should be a list --- python/cudf/cudf/_lib/parquet.pyx | 7 ++++++- python/cudf/cudf/io/parquet.py | 4 +++- 2 files changed, 9 insertions(+), 2 deletions(-) diff --git a/python/cudf/cudf/_lib/parquet.pyx b/python/cudf/cudf/_lib/parquet.pyx index f3f385a8e24..8578e77310f 100644 --- a/python/cudf/cudf/_lib/parquet.pyx +++ b/python/cudf/cudf/_lib/parquet.pyx @@ -23,6 +23,7 @@ from cudf.api.types import ( is_categorical_dtype, is_decimal_dtype, is_list_dtype, + is_list_like, is_struct_dtype, ) from cudf.utils.dtypes import np_to_pa_dtype @@ -365,7 +366,11 @@ cpdef write_parquet( partitions.push_back(cudf_io_types.partition_info(part[0], part[1])) args.set_partitions(move(partitions)) if metadata_file_path is not None: - c_column_chunks_file_paths.push_back(str.encode(metadata_file_path)) + if is_list_like(metadata_file_path): + for path in metadata_file_path: + c_column_chunks_file_paths.push_back(str.encode(path)) + else: + c_column_chunks_file_paths.push_back(str.encode(metadata_file_path)) args.set_column_chunks_file_paths(move(c_column_chunks_file_paths)) if row_group_size_bytes is not None: args.set_row_group_size_bytes(row_group_size_bytes) diff --git a/python/cudf/cudf/io/parquet.py b/python/cudf/cudf/io/parquet.py index 366fc39cc37..acaec92204a 100644 --- a/python/cudf/cudf/io/parquet.py +++ b/python/cudf/cudf/io/parquet.py @@ -159,6 +159,7 @@ def write_to_dataset( part_names = part_names.to_pandas().to_frame(index=False) full_paths = [] + metadata_file_paths = [] for keys in part_names.itertuples(index=False): subdir = fs.sep.join( [f"{name}={val}" for name, val in zip(partition_cols, keys)] @@ -168,6 +169,7 @@ def write_to_dataset( filename = filename or uuid4().hex + ".parquet" full_path = fs.sep.join([prefix, filename]) full_paths.append(full_path) + metadata_file_paths.append(fs.sep.join([subdir, filename])) with ExitStack() as stack: open_files = [ @@ -181,7 +183,7 @@ def write_to_dataset( grouped_df, file_objs, index=preserve_index, - metadata_file_path=fs.sep.join([subdir, filename]), + metadata_file_path=metadata_file_paths, partitions_info=partitions_info, **kwargs, ) From 2442be8fcec0049fd0fe53b17313efd6a30bb8ee Mon Sep 17 00:00:00 2001 From: Devavret Makkar Date: Sat, 18 Dec 2021 03:48:34 +0530 Subject: [PATCH 44/71] Make all sinks work again --- python/cudf/cudf/_lib/io/utils.pyx | 58 ++++++++++++------------------ 1 file changed, 23 insertions(+), 35 deletions(-) diff --git a/python/cudf/cudf/_lib/io/utils.pyx b/python/cudf/cudf/_lib/io/utils.pyx index f490e0dbc5e..5ec7c09858a 100644 --- a/python/cudf/cudf/_lib/io/utils.pyx +++ b/python/cudf/cudf/_lib/io/utils.pyx @@ -82,21 +82,22 @@ cdef source_info make_source_info(list src) except*: cdef sink_info make_sinks_info(list src, vector[unique_ptr[data_sink]] & sink) except*: cdef vector[data_sink *] data_sinks cdef vector[string] paths - # TODO: re-enable after paths splits is known to work - # if isinstance(src[0], io.StringIO): - # for s in src: - # sink.push_back(new iobase_data_sink(src)) - # data_sinks.push_back(sink.back().get()) - # return sink_info(data_sinks) - # elif isinstance(src[0], io.TextIOBase): - # # Files opened in text mode expect writes to be str rather than bytes, - # # which requires conversion from utf-8. If the underlying buffer is - # # utf-8, we can bypass this conversion by writing directly to it. - # if codecs.lookup(src.encoding).name not in {"utf-8", "ascii"}: - # raise NotImplementedError(f"Unsupported encoding {src.encoding}") - # sink.reset(new iobase_data_sink(src.buffer)) - # return sink_info(sink.get()) - if isinstance(src[0], io.IOBase): + if isinstance(src[0], io.StringIO): + for s in src: + sink.push_back(unique_ptr[data_sink](new iobase_data_sink(s))) + data_sinks.push_back(sink.back().get()) + return sink_info(data_sinks) + elif isinstance(src[0], io.TextIOBase): + for s in src: + # Files opened in text mode expect writes to be str rather than bytes, + # which requires conversion from utf-8. If the underlying buffer is + # utf-8, we can bypass this conversion by writing directly to it. + if codecs.lookup(s.encoding).name not in {"utf-8", "ascii"}: + raise NotImplementedError(f"Unsupported encoding {s.encoding}") + sink.push_back(unique_ptr[data_sink](new iobase_data_sink(s.buffer))) + data_sinks.push_back(sink.back().get()) + return sink_info(data_sinks) + elif isinstance(src[0], io.IOBase): for s in src: sink.push_back(unique_ptr[data_sink](new iobase_data_sink(s))) data_sinks.push_back(sink.back().get()) @@ -105,29 +106,16 @@ cdef sink_info make_sinks_info(list src, vector[unique_ptr[data_sink]] & sink) e for s in src: paths.push_back( os.path.expanduser(s).encode()) return sink_info(move(paths)) - # else: - # raise TypeError("Unrecognized input type: {}".format(type(src))) + else: + raise TypeError("Unrecognized input type: {}".format(type(src))) cdef sink_info make_sink_info(src, unique_ptr[data_sink] & sink) except*: - if isinstance(src, io.StringIO): - sink.reset(new iobase_data_sink(src)) - return sink_info(sink.get()) - elif isinstance(src, io.TextIOBase): - # Files opened in text mode expect writes to be str rather than bytes, - # which requires conversion from utf-8. If the underlying buffer is - # utf-8, we can bypass this conversion by writing directly to it. - if codecs.lookup(src.encoding).name not in {"utf-8", "ascii"}: - raise NotImplementedError(f"Unsupported encoding {src.encoding}") - sink.reset(new iobase_data_sink(src.buffer)) - return sink_info(sink.get()) - elif isinstance(src, io.IOBase): - sink.reset(new iobase_data_sink(src)) - return sink_info(sink.get()) - elif isinstance(src, (basestring, os.PathLike)): - return sink_info( os.path.expanduser(src).encode()) - else: - raise TypeError("Unrecognized input type: {}".format(type(src))) + cdef vector[unique_ptr[data_sink]] datasinks + cdef sink_info info = make_sinks_info([src], datasinks) + if not datasinks.empty(): + sink.swap(datasinks[0]) + return info # Adapts a python io.IOBase object as a libcudf++ IO data_sink. This lets you From 48354e96fb453f8542ddfda505413ebcd9b9454c Mon Sep 17 00:00:00 2001 From: Devavret Makkar Date: Sat, 18 Dec 2021 05:04:39 +0530 Subject: [PATCH 45/71] Consolidate common args and save some lines --- python/cudf/cudf/io/parquet.py | 32 ++++++++++++-------------------- 1 file changed, 12 insertions(+), 20 deletions(-) diff --git a/python/cudf/cudf/io/parquet.py b/python/cudf/cudf/io/parquet.py index acaec92204a..14d1c9bcaf1 100644 --- a/python/cudf/cudf/io/parquet.py +++ b/python/cudf/cudf/io/parquet.py @@ -50,6 +50,16 @@ def _write_parquet( ioutils.get_writer_filepath_or_buffer(path, mode="wb", **kwargs) for path in paths ] + common_args = { + "index": index, + "compression": compression, + "statistics": statistics, + "metadata_file_path": metadata_file_path, + "int96_timestamps": int96_timestamps, + "row_group_size_bytes": row_group_size_bytes, + "row_group_size_rows": row_group_size_rows, + "partitions_info": partitions_info, + } if all([ioutils.is_fsspec_open_file(buf) for buf in paths_or_bufs]): with ExitStack() as stack: fsspec_objs = [ @@ -59,29 +69,11 @@ def _write_parquet( ioutils.get_IOBase_writer(file_obj) for file_obj in fsspec_objs ] write_parquet_res = libparquet.write_parquet( - df, - filepaths_or_buffers=file_objs, - index=index, - compression=compression, - statistics=statistics, - metadata_file_path=metadata_file_path, - int96_timestamps=int96_timestamps, - row_group_size_bytes=row_group_size_bytes, - row_group_size_rows=row_group_size_rows, - partitions_info=partitions_info, + df, filepaths_or_buffers=file_objs, **common_args ) else: write_parquet_res = libparquet.write_parquet( - df, - filepaths_or_buffers=paths_or_bufs, - index=index, - compression=compression, - statistics=statistics, - metadata_file_path=metadata_file_path, - int96_timestamps=int96_timestamps, - row_group_size_bytes=row_group_size_bytes, - row_group_size_rows=row_group_size_rows, - partitions_info=partitions_info, + df, filepaths_or_buffers=paths_or_bufs, **common_args ) return write_parquet_res From 72b1e81ada4cd316e33e81caca89d9a89d01a0e2 Mon Sep 17 00:00:00 2001 From: Devavret Makkar Date: Sat, 18 Dec 2021 05:38:55 +0530 Subject: [PATCH 46/71] write_to_dataset cleanups: - No longer need merge metadata in write_to_dataset - Use kwargs to reduce subsequent calls --- python/cudf/cudf/io/parquet.py | 57 +++++++++------------------------- 1 file changed, 14 insertions(+), 43 deletions(-) diff --git a/python/cudf/cudf/io/parquet.py b/python/cudf/cudf/io/parquet.py index 14d1c9bcaf1..e2f8675af9d 100644 --- a/python/cudf/cudf/io/parquet.py +++ b/python/cudf/cudf/io/parquet.py @@ -129,7 +129,6 @@ def write_to_dataset( fs = ioutils._ensure_filesystem(fs, root_path, **kwargs) fs.mkdirs(root_path, exist_ok=True) - metadata = [] if partition_cols is not None and len(partition_cols) > 0: @@ -161,55 +160,27 @@ def write_to_dataset( filename = filename or uuid4().hex + ".parquet" full_path = fs.sep.join([prefix, filename]) full_paths.append(full_path) - metadata_file_paths.append(fs.sep.join([subdir, filename])) - - with ExitStack() as stack: - open_files = [ - stack.enter_context(fs.open(path, mode="wb")) - for path in full_paths - ] - file_objs = [ioutils.get_IOBase_writer(fil) for fil in open_files] if return_metadata: - metadata.append( - _write_parquet( - grouped_df, - file_objs, - index=preserve_index, - metadata_file_path=metadata_file_paths, - partitions_info=partitions_info, - **kwargs, - ) - ) - else: - _write_parquet( - grouped_df, - file_objs, - index=preserve_index, - partitions_info=partitions_info, - **kwargs, - ) + metadata_file_paths.append(fs.sep.join([subdir, filename])) + + if return_metadata: + kwargs["metadata_file_path"] = metadata_file_paths + metadata = _write_parquet( + grouped_df, + full_paths, + index=preserve_index, + partitions_info=partitions_info, + **kwargs, + ) else: filename = filename or uuid4().hex + ".parquet" full_path = fs.sep.join([root_path, filename]) if return_metadata: - metadata.append( - df.to_parquet( - full_path, - index=preserve_index, - metadata_file_path=filename, - **kwargs, - ) - ) - else: - df.to_parquet(full_path, index=preserve_index, **kwargs) + kwargs["metadata_file_path"] = filename + metadata = df.to_parquet(full_path, index=preserve_index, **kwargs) - if metadata: - return ( - merge_parquet_filemetadata(metadata) - if len(metadata) > 1 - else metadata[0] - ) + return metadata @ioutils.doc_read_parquet_metadata() From e84fc1c8880813d76a4d66334c8670888310f289 Mon Sep 17 00:00:00 2001 From: Devavret Makkar Date: Tue, 21 Dec 2021 00:04:30 +0530 Subject: [PATCH 47/71] Write_to_dataset calls back to_parquet. Move offset to ranges calculation to to_parquet --- python/cudf/cudf/io/parquet.py | 19 ++++++++++++------- 1 file changed, 12 insertions(+), 7 deletions(-) diff --git a/python/cudf/cudf/io/parquet.py b/python/cudf/cudf/io/parquet.py index e2f8675af9d..1442f151075 100644 --- a/python/cudf/cudf/io/parquet.py +++ b/python/cudf/cudf/io/parquet.py @@ -142,10 +142,6 @@ def write_to_dataset( if not preserve_index: grouped_df.reset_index(drop=True, inplace=True) grouped_df.drop(columns=partition_cols, inplace=True) - partitions_info = [ - (part_offsets[i], part_offsets[i + 1] - part_offsets[i]) - for i in range(0, len(part_offsets) - 1) - ] # Copy the entire keys df in one operation rather than using iloc part_names = part_names.to_pandas().to_frame(index=False) @@ -165,11 +161,11 @@ def write_to_dataset( if return_metadata: kwargs["metadata_file_path"] = metadata_file_paths - metadata = _write_parquet( + metadata = to_parquet( grouped_df, full_paths, index=preserve_index, - partitions_info=partitions_info, + partition_offsets=part_offsets, **kwargs, ) @@ -736,9 +732,18 @@ def to_parquet( return # TODO: re-enable non-partitioned case + if partition_offsets: + kwargs["partitions_info"] = [ + ( + partition_offsets[i], + partition_offsets[i + 1] - partition_offsets[i], + ) + for i in range(0, len(partition_offsets) - 1) + ] + return _write_parquet( df, - paths=[path], + paths=path if is_list_like(path) else [path], compression=compression, index=index, statistics=statistics, From 96139d4a998b388561994ee585f203c15a3eb66e Mon Sep 17 00:00:00 2001 From: Devavret Makkar Date: Tue, 21 Dec 2021 02:24:19 +0530 Subject: [PATCH 48/71] Passthrough other args with partition_cols instead of ignoring Various checks/warnings/exceptions --- python/cudf/cudf/io/parquet.py | 47 ++++++++++++++++++++-------------- 1 file changed, 28 insertions(+), 19 deletions(-) diff --git a/python/cudf/cudf/io/parquet.py b/python/cudf/cudf/io/parquet.py index 1442f151075..43a2a649632 100644 --- a/python/cudf/cudf/io/parquet.py +++ b/python/cudf/cudf/io/parquet.py @@ -32,19 +32,16 @@ def _write_parquet( partitions_info=None, **kwargs, ): - # TODO: exceptions: - # If paths are multiple then partitions_info are required. - # If partitions_info are specified then paths should be list like. - # In either case, both should be of same length. - # Partition offsets should be a list of 2-tuples - # if is_list_like(paths) and len(paths) > 1: - # if partitions_info is None: - # ValueError("partition info is required for multiple paths") - # elif not is_list_like(partitions_info): - # ValueError("partition info must be list-like") - - # if not len(paths) == len(partitions_info): - # ValueError("") + if is_list_like(paths) and len(paths) > 1: + if partitions_info is None: + ValueError("partition info is required for multiple paths") + elif not is_list_like(partitions_info): + ValueError("partition info must be list-like for multiple paths") + elif not len(paths) == len(partitions_info): + ValueError("partitions_info and paths must be of same size") + if is_list_like(partitions_info) and len(partitions_info) > 1: + if not is_list_like(paths): + ValueError("paths must be list-like when partitions_info provided") paths_or_bufs = [ ioutils.get_writer_filepath_or_buffer(path, mode="wb", **kwargs) @@ -716,11 +713,21 @@ def to_parquet( + "supported by the gpu accelerated parquet writer" ) - # TODO: propagate compression, statistics, int96, row_group_size_bytes, - # row_group_size_rows. - # TODO: warn against metadata_file_path being provided when partition - # cols specified if partition_cols: + if metadata_file_path is not None: + warnings.warn( + "metadata_file_path will be ignored/overwritten when " + "partition_cols are provided" + ) + kwargs.update( + { + "compression": compression, + "statistics": statistics, + "int96_timestamps": int96_timestamps, + "row_group_size_bytes": row_group_size_bytes, + "row_group_size_rows": row_group_size_rows, + } + ) write_to_dataset( df, filename=partition_file_name, @@ -731,7 +738,6 @@ def to_parquet( ) return - # TODO: re-enable non-partitioned case if partition_offsets: kwargs["partitions_info"] = [ ( @@ -755,7 +761,10 @@ def to_parquet( ) else: - # TODO: error if partition_offsets is specified without cudf engine + if partition_offsets is not None: + warnings.warn( + "partition_offsets will be ignored when engine is not cudf" + ) # If index is empty set it to the expected default value of True if index is None: From f6d4175e5e6997644760875e40c56efe34c96cf6 Mon Sep 17 00:00:00 2001 From: Devavret Makkar Date: Tue, 21 Dec 2021 06:00:24 +0530 Subject: [PATCH 49/71] Fix style issues in python/cython --- python/cudf/cudf/_lib/io/utils.pxd | 3 ++- python/cudf/cudf/_lib/io/utils.pyx | 15 ++++++++++----- python/cudf/cudf/_lib/parquet.pyx | 16 ++++++++++++---- python/cudf/cudf/io/parquet.py | 1 - 4 files changed, 24 insertions(+), 11 deletions(-) diff --git a/python/cudf/cudf/_lib/io/utils.pxd b/python/cudf/cudf/_lib/io/utils.pxd index 1c0926b90c6..af1f2521d4a 100644 --- a/python/cudf/cudf/_lib/io/utils.pxd +++ b/python/cudf/cudf/_lib/io/utils.pxd @@ -13,7 +13,8 @@ from cudf._lib.cpp.io.types cimport ( cdef source_info make_source_info(list src) except* -cdef sink_info make_sinks_info(list src, vector[unique_ptr[data_sink]] & data) except* +cdef sink_info make_sinks_info( + list src, vector[unique_ptr[data_sink]] & data) except* cdef sink_info make_sink_info(src, unique_ptr[data_sink] & data) except* cdef update_struct_field_names( table, diff --git a/python/cudf/cudf/_lib/io/utils.pyx b/python/cudf/cudf/_lib/io/utils.pyx index 5ec7c09858a..f3aa0cfdbc9 100644 --- a/python/cudf/cudf/_lib/io/utils.pyx +++ b/python/cudf/cudf/_lib/io/utils.pyx @@ -79,7 +79,9 @@ cdef source_info make_source_info(list src) except*: return source_info(c_host_buffers) # Converts the Python sink input to libcudf++ IO sink_info. -cdef sink_info make_sinks_info(list src, vector[unique_ptr[data_sink]] & sink) except*: +cdef sink_info make_sinks_info( + list src, vector[unique_ptr[data_sink]] & sink +) except*: cdef vector[data_sink *] data_sinks cdef vector[string] paths if isinstance(src[0], io.StringIO): @@ -89,12 +91,15 @@ cdef sink_info make_sinks_info(list src, vector[unique_ptr[data_sink]] & sink) e return sink_info(data_sinks) elif isinstance(src[0], io.TextIOBase): for s in src: - # Files opened in text mode expect writes to be str rather than bytes, - # which requires conversion from utf-8. If the underlying buffer is - # utf-8, we can bypass this conversion by writing directly to it. + # Files opened in text mode expect writes to be str rather than + # bytes, which requires conversion from utf-8. If the underlying + # buffer is utf-8, we can bypass this conversion by writing + # directly to it. if codecs.lookup(s.encoding).name not in {"utf-8", "ascii"}: raise NotImplementedError(f"Unsupported encoding {s.encoding}") - sink.push_back(unique_ptr[data_sink](new iobase_data_sink(s.buffer))) + sink.push_back( + unique_ptr[data_sink](new iobase_data_sink(s.buffer)) + ) data_sinks.push_back(sink.back().get()) return sink_info(data_sinks) elif isinstance(src[0], io.IOBase): diff --git a/python/cudf/cudf/_lib/parquet.pyx b/python/cudf/cudf/_lib/parquet.pyx index 8578e77310f..ef3c2dccb00 100644 --- a/python/cudf/cudf/_lib/parquet.pyx +++ b/python/cudf/cudf/_lib/parquet.pyx @@ -302,7 +302,9 @@ cpdef write_parquet( cdef vector[map[string, string]] user_data cdef table_view tv cdef vector[unique_ptr[cudf_io_types.data_sink]] _data_sinks - cdef cudf_io_types.sink_info sink = make_sinks_info(filepaths_or_buffers, _data_sinks) + cdef cudf_io_types.sink_info sink = make_sinks_info( + filepaths_or_buffers, _data_sinks + ) if index is True or ( index is None and not isinstance(table._index, cudf.RangeIndex) @@ -333,7 +335,9 @@ cpdef write_parquet( cdef map[string, string] tmp_user_data if partitions_info is not None: for start_row, num_row in partitions_info: - partitioned_df = table.iloc[start_row: start_row+ num_row].copy(deep=False) + partitioned_df = table.iloc[start_row: start_row + num_row].copy( + deep=False + ) pandas_metadata = generate_pandas_metadata(partitioned_df, index) tmp_user_data[str.encode("pandas")] = str.encode(pandas_metadata) user_data.push_back(tmp_user_data) @@ -363,14 +367,18 @@ cpdef write_parquet( ) if partitions_info is not None: for part in partitions_info: - partitions.push_back(cudf_io_types.partition_info(part[0], part[1])) + partitions.push_back( + cudf_io_types.partition_info(part[0], part[1]) + ) args.set_partitions(move(partitions)) if metadata_file_path is not None: if is_list_like(metadata_file_path): for path in metadata_file_path: c_column_chunks_file_paths.push_back(str.encode(path)) else: - c_column_chunks_file_paths.push_back(str.encode(metadata_file_path)) + c_column_chunks_file_paths.push_back( + str.encode(metadata_file_path) + ) args.set_column_chunks_file_paths(move(c_column_chunks_file_paths)) if row_group_size_bytes is not None: args.set_row_group_size_bytes(row_group_size_bytes) diff --git a/python/cudf/cudf/io/parquet.py b/python/cudf/cudf/io/parquet.py index 43a2a649632..6c08550e8fc 100644 --- a/python/cudf/cudf/io/parquet.py +++ b/python/cudf/cudf/io/parquet.py @@ -2,7 +2,6 @@ import io import json -from typing import ValuesView import warnings from collections import defaultdict from contextlib import ExitStack From 833bc8879d7dfede0083034dcad9345f1e3c59d1 Mon Sep 17 00:00:00 2001 From: Devavret Makkar Date: Thu, 30 Dec 2021 02:09:44 +0530 Subject: [PATCH 50/71] allowing cython ParquetWriter to call write with partitions --- python/cudf/cudf/_lib/cpp/io/parquet.pxd | 4 +++ python/cudf/cudf/_lib/parquet.pyx | 32 ++++++++++++++++-------- 2 files changed, 26 insertions(+), 10 deletions(-) diff --git a/python/cudf/cudf/_lib/cpp/io/parquet.pxd b/python/cudf/cudf/_lib/cpp/io/parquet.pxd index 07b312361f2..93780412f37 100644 --- a/python/cudf/cudf/_lib/cpp/io/parquet.pxd +++ b/python/cudf/cudf/_lib/cpp/io/parquet.pxd @@ -207,6 +207,10 @@ cdef extern from "cudf/io/parquet.hpp" namespace "cudf::io" nogil: parquet_chunked_writer& write( cudf_table_view.table_view table_, ) except+ + parquet_chunked_writer& write( + cudf_table_view.table_view table_, + vector[cudf_io_types.partition_info] partitions, + ) except+ unique_ptr[vector[uint8_t]] close( vector[string] column_chunks_file_paths, ) except+ diff --git a/python/cudf/cudf/_lib/parquet.pyx b/python/cudf/cudf/_lib/parquet.pyx index ef3c2dccb00..3fe342f8287 100644 --- a/python/cudf/cudf/_lib/parquet.pyx +++ b/python/cudf/cudf/_lib/parquet.pyx @@ -410,23 +410,26 @@ cdef class ParquetWriter: cdef unique_ptr[cpp_parquet_chunked_writer] writer cdef unique_ptr[table_input_metadata] tbl_meta cdef cudf_io_types.sink_info sink - cdef unique_ptr[cudf_io_types.data_sink] _data_sink + cdef vector[unique_ptr[cudf_io_types.data_sink]] _data_sink cdef cudf_io_types.statistics_freq stat_freq cdef cudf_io_types.compression_type comp_type cdef object index - def __cinit__(self, object path, object index=None, + def __cinit__(self, object filepaths_or_buffers, object index=None, object compression=None, str statistics="ROWGROUP"): - self.sink = make_sink_info(path, self._data_sink) + self.sink = make_sinks_info(filepaths_or_buffers, self._data_sink) self.stat_freq = _get_stat_freq(statistics) self.comp_type = _get_comp_type(compression) self.index = index self.initialized = False - def write_table(self, table): + def write_table(self, table, object partitions_info=None): """ Writes a single table to the file """ if not self.initialized: - self._initialize_chunked_state(table) + self._initialize_chunked_state( + table, + num_partitions=len(partitions_info) if partitions_info else 1 + ) cdef table_view tv if self.index is not False and ( @@ -436,8 +439,15 @@ cdef class ParquetWriter: else: tv = table_view_from_table(table, ignore_index=True) + cdef vector[cudf_io_types.partition_info] partitions + if partitions_info is not None: + for part in partitions_info: + partitions.push_back( + cudf_io_types.partition_info(part[0], part[1]) + ) + with nogil: - self.writer.get()[0].write(tv) + self.writer.get()[0].write(tv, partitions) def close(self, object metadata_file_path=None): cdef unique_ptr[vector[uint8_t]] out_metadata_c @@ -465,7 +475,7 @@ cdef class ParquetWriter: def __dealloc__(self): self.close() - def _initialize_chunked_state(self, table): + def _initialize_chunked_state(self, table, num_partitions=1): """ Prepares all the values required to build the chunked_parquet_writer_options and creates a writer""" cdef table_view tv @@ -498,10 +508,12 @@ cdef class ParquetWriter: table[name]._column, self.tbl_meta.get().column_metadata[i] ) - pandas_metadata = generate_pandas_metadata(table, self.index) + index = False if isinstance(table._index, cudf.RangeIndex) else self.index + pandas_metadata = generate_pandas_metadata(table, index) + cdef map[string, string] tmp_user_data + tmp_user_data[str.encode("pandas")] = str.encode(pandas_metadata) cdef vector[map[string, string]] user_data - user_data.resize(1) - user_data.back()[str.encode("pandas")] = str.encode(pandas_metadata) + user_data = vector[map[string, string]](num_partitions, tmp_user_data) cdef chunked_parquet_writer_options args with nogil: From 15557b7aee98a13f0f896193b11ba8deb8cfd868 Mon Sep 17 00:00:00 2001 From: Devavret Makkar Date: Thu, 30 Dec 2021 02:10:42 +0530 Subject: [PATCH 51/71] First working poc of ParquetWriter class that can dynamically handle new keys --- python/cudf/cudf/io/parquet.py | 115 ++++++++++++++++++++++++++++++++- 1 file changed, 114 insertions(+), 1 deletion(-) diff --git a/python/cudf/cudf/io/parquet.py b/python/cudf/cudf/io/parquet.py index 6c08550e8fc..097d46c4883 100644 --- a/python/cudf/cudf/io/parquet.py +++ b/python/cudf/cudf/io/parquet.py @@ -791,7 +791,120 @@ def merge_parquet_filemetadata(filemetadata_list): return libparquet.merge_filemetadata(filemetadata_list) -ParquetWriter = libparquet.ParquetWriter +def _get_partitioned( + df, + root_path, + partition_cols, + filename=None, + fs=None, + return_metadata=False, + **kwargs, +): + fs = ioutils._ensure_filesystem(fs, root_path, **kwargs) + fs.mkdirs(root_path, exist_ok=True) + data_cols = df.columns.drop(partition_cols) + if len(data_cols) == 0: + raise ValueError("No data left to save outside partition columns") + + part_names, part_offsets, _, grouped_df = df.groupby( + partition_cols + )._grouped() + # if not preserve_index: + # grouped_df.reset_index(drop=True, inplace=True) + grouped_df.drop(columns=partition_cols, inplace=True) + # Copy the entire keys df in one operation rather than using iloc + part_names = part_names.to_pandas().to_frame(index=False) + + full_paths = [] + metadata_file_paths = [] + for keys in part_names.itertuples(index=False): + subdir = fs.sep.join( + [f"{name}={val}" for name, val in zip(partition_cols, keys)] + ) + prefix = fs.sep.join([root_path, subdir]) + fs.mkdirs(prefix, exist_ok=True) + filename = filename or uuid4().hex + ".parquet" + full_path = fs.sep.join([prefix, filename]) + full_paths.append(full_path) + if return_metadata: + metadata_file_paths.append(fs.sep.join([subdir, filename])) + + return full_paths, grouped_df, part_offsets, filename + + +class ParquetWriter: + def __init__(self, path, partition_cols=None, fs=None) -> None: + # Holds in self: + # 1. Collection of libparquet.ParquetWriter, each responsible for own set of partition keys + # 2. + self.path = path + self.partition_cols = partition_cols + self._chunked_writers = [] + self.path_cw_map = {} + self.fs = fs + self.filename = None + pass + + def write_table(self, df): + # Get partitions - paths (keys), grouped_df, offsets + # Add to libparquet.ParquetWriter collection + # if some paths match other writer then remove + paths, grouped_df, offsets, self.filename = _get_partitioned( + df, + self.path, + self.partition_cols, + fs=self.fs, + filename=self.filename, + ) + + existing_cw_batch = defaultdict(dict) + new_cw_paths = [] + + def pairwise(iterable): + it = iter(iterable) + a = next(it, None) + for b in it: + yield (a, b - a) + a = b + + for path, part_info in zip(paths, pairwise(offsets)): # and bounds + if path in self.path_cw_map: # path is a currently open file + cw_idx = self.path_cw_map[path] + existing_cw_batch[cw_idx][path] = part_info + else: # path not currently handled by any chunked writer + new_cw_paths.append((path, part_info)) + + # Write out the parts of the grouped_df currently handled by existing cws + for batch_cw_path_list in existing_cw_batch.items(): + cw_idx = batch_cw_path_list[0] + cw = self._chunked_writers[cw_idx][0] + # match found paths with this cw's paths and nullify partition info for ones not in this batch + this_cw_part_info = [] + for path in self._chunked_writers[cw_idx][1]: + if path in batch_cw_path_list[1]: + this_cw_part_info.append(batch_cw_path_list[1][path]) + else: + this_cw_part_info.append((0, 0)) + + cw.write_table(grouped_df, this_cw_part_info) + + # Create new cw for unhandled paths encountered in this write_table + new_paths = [path for path, _ in new_cw_paths] + self._chunked_writers.append( + (libparquet.ParquetWriter(new_paths), new_paths) + ) + new_cw_idx = len(self._chunked_writers) - 1 + for path in new_paths: + self.path_cw_map[path] = new_cw_idx + part_info = [info for _, info in new_cw_paths] + self._chunked_writers[-1][0].write_table(grouped_df, part_info) + + def close(self): + for cw, _ in self._chunked_writers: + cw.close() + + def __del__(self): + self.close() def _check_decimal128_type(arrow_type): From 879294c4d526028957ca9e9728e6d383bc9ddbf2 Mon Sep 17 00:00:00 2001 From: Devavret Makkar Date: Thu, 30 Dec 2021 03:25:16 +0530 Subject: [PATCH 52/71] code cleanups to be more pythonic --- python/cudf/cudf/io/parquet.py | 28 +++++++++++++--------------- 1 file changed, 13 insertions(+), 15 deletions(-) diff --git a/python/cudf/cudf/io/parquet.py b/python/cudf/cudf/io/parquet.py index 097d46c4883..c29b2d54e95 100644 --- a/python/cudf/cudf/io/parquet.py +++ b/python/cudf/cudf/io/parquet.py @@ -834,12 +834,13 @@ def _get_partitioned( class ParquetWriter: def __init__(self, path, partition_cols=None, fs=None) -> None: - # Holds in self: - # 1. Collection of libparquet.ParquetWriter, each responsible for own set of partition keys - # 2. self.path = path self.partition_cols = partition_cols + # Collection of `libparquet.ParquetWriter`s, and the corresponding + # partition_col values they're responsible for self._chunked_writers = [] + # Map of partition_col values to their libparquet.ParquetWriter's index + # in self._chunked_writers for reverse lookup self.path_cw_map = {} self.fs = fs self.filename = None @@ -867,25 +868,23 @@ def pairwise(iterable): yield (a, b - a) a = b - for path, part_info in zip(paths, pairwise(offsets)): # and bounds + for path, part_info in zip(paths, pairwise(offsets)): if path in self.path_cw_map: # path is a currently open file cw_idx = self.path_cw_map[path] existing_cw_batch[cw_idx][path] = part_info else: # path not currently handled by any chunked writer new_cw_paths.append((path, part_info)) - # Write out the parts of the grouped_df currently handled by existing cws + # Write out the parts of grouped_df currently handled by existing cw's for batch_cw_path_list in existing_cw_batch.items(): cw_idx = batch_cw_path_list[0] cw = self._chunked_writers[cw_idx][0] - # match found paths with this cw's paths and nullify partition info for ones not in this batch - this_cw_part_info = [] - for path in self._chunked_writers[cw_idx][1]: - if path in batch_cw_path_list[1]: - this_cw_part_info.append(batch_cw_path_list[1][path]) - else: - this_cw_part_info.append((0, 0)) - + # match found paths with this cw's paths and nullify partition info + # for partition_col values not in this batch + this_cw_part_info = [ + batch_cw_path_list[1].get(path, (0, 0)) + for path in self._chunked_writers[cw_idx][1] + ] cw.write_table(grouped_df, this_cw_part_info) # Create new cw for unhandled paths encountered in this write_table @@ -894,8 +893,7 @@ def pairwise(iterable): (libparquet.ParquetWriter(new_paths), new_paths) ) new_cw_idx = len(self._chunked_writers) - 1 - for path in new_paths: - self.path_cw_map[path] = new_cw_idx + self.path_cw_map.update({k: new_cw_idx for k in new_paths}) part_info = [info for _, info in new_cw_paths] self._chunked_writers[-1][0].write_table(grouped_df, part_info) From 2bcf62cca95b0bdbae1542178c2d83390a2f1bd0 Mon Sep 17 00:00:00 2001 From: Devavret Makkar Date: Thu, 30 Dec 2021 04:11:19 +0530 Subject: [PATCH 53/71] enable regular old non-partitioned writing --- python/cudf/cudf/io/parquet.py | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/python/cudf/cudf/io/parquet.py b/python/cudf/cudf/io/parquet.py index c29b2d54e95..7bd0ddf8abe 100644 --- a/python/cudf/cudf/io/parquet.py +++ b/python/cudf/cudf/io/parquet.py @@ -844,12 +844,16 @@ def __init__(self, path, partition_cols=None, fs=None) -> None: self.path_cw_map = {} self.fs = fs self.filename = None - pass + if partition_cols is None: + self._chunked_writers.append( + (libparquet.ParquetWriter([path]), []) + ) def write_table(self, df): - # Get partitions - paths (keys), grouped_df, offsets - # Add to libparquet.ParquetWriter collection - # if some paths match other writer then remove + if self.partition_cols is None: + self._chunked_writers[0][0].write_table(df) + return + paths, grouped_df, offsets, self.filename = _get_partitioned( df, self.path, From cf13e815d57e465463e108b3b85b39d4d18aaac5 Mon Sep 17 00:00:00 2001 From: Devavret Makkar Date: Thu, 30 Dec 2021 05:34:45 +0530 Subject: [PATCH 54/71] Enable index arg and passthrough compression and statistics --- python/cudf/cudf/io/parquet.py | 28 +++++++++++++++++++++++----- 1 file changed, 23 insertions(+), 5 deletions(-) diff --git a/python/cudf/cudf/io/parquet.py b/python/cudf/cudf/io/parquet.py index 7bd0ddf8abe..7286102e90f 100644 --- a/python/cudf/cudf/io/parquet.py +++ b/python/cudf/cudf/io/parquet.py @@ -797,6 +797,7 @@ def _get_partitioned( partition_cols, filename=None, fs=None, + preserve_index=False, return_metadata=False, **kwargs, ): @@ -809,8 +810,8 @@ def _get_partitioned( part_names, part_offsets, _, grouped_df = df.groupby( partition_cols )._grouped() - # if not preserve_index: - # grouped_df.reset_index(drop=True, inplace=True) + if not preserve_index: + grouped_df.reset_index(drop=True, inplace=True) grouped_df.drop(columns=partition_cols, inplace=True) # Copy the entire keys df in one operation rather than using iloc part_names = part_names.to_pandas().to_frame(index=False) @@ -833,8 +834,21 @@ def _get_partitioned( class ParquetWriter: - def __init__(self, path, partition_cols=None, fs=None) -> None: + def __init__( + self, + path, + index=None, + compression=None, + statistics="ROWGROUP", + partition_cols=None, + fs=None, + ) -> None: self.path = path + self.common_args = { + "index": index, + "compression": compression, + "statistics": statistics, + } self.partition_cols = partition_cols # Collection of `libparquet.ParquetWriter`s, and the corresponding # partition_col values they're responsible for @@ -846,7 +860,7 @@ def __init__(self, path, partition_cols=None, fs=None) -> None: self.filename = None if partition_cols is None: self._chunked_writers.append( - (libparquet.ParquetWriter([path]), []) + (libparquet.ParquetWriter([path], **self.common_args), []) ) def write_table(self, df): @@ -858,6 +872,7 @@ def write_table(self, df): df, self.path, self.partition_cols, + preserve_index=self.common_args["index"], fs=self.fs, filename=self.filename, ) @@ -894,7 +909,10 @@ def pairwise(iterable): # Create new cw for unhandled paths encountered in this write_table new_paths = [path for path, _ in new_cw_paths] self._chunked_writers.append( - (libparquet.ParquetWriter(new_paths), new_paths) + ( + libparquet.ParquetWriter(new_paths, **self.common_args), + new_paths, + ) ) new_cw_idx = len(self._chunked_writers) - 1 self.path_cw_map.update({k: new_cw_idx for k in new_paths}) From 7e57eecd56959408f958e8785fa0bf612787685c Mon Sep 17 00:00:00 2001 From: Devavret Makkar Date: Fri, 31 Dec 2021 05:26:17 +0530 Subject: [PATCH 55/71] ability to return metadata --- python/cudf/cudf/_lib/parquet.pyx | 8 +++- python/cudf/cudf/io/parquet.py | 66 ++++++++++++++++++++++++------- 2 files changed, 58 insertions(+), 16 deletions(-) diff --git a/python/cudf/cudf/_lib/parquet.pyx b/python/cudf/cudf/_lib/parquet.pyx index 3fe342f8287..a7d4cc0f93b 100644 --- a/python/cudf/cudf/_lib/parquet.pyx +++ b/python/cudf/cudf/_lib/parquet.pyx @@ -458,7 +458,13 @@ cdef class ParquetWriter: # Update metadata-collection options if metadata_file_path is not None: - column_chunks_file_paths.push_back(str.encode(metadata_file_path)) + if is_list_like(metadata_file_path): + for path in metadata_file_path: + column_chunks_file_paths.push_back(str.encode(path)) + else: + column_chunks_file_paths.push_back( + str.encode(metadata_file_path) + ) with nogil: out_metadata_c = move( diff --git a/python/cudf/cudf/io/parquet.py b/python/cudf/cudf/io/parquet.py index 7286102e90f..7cd68eb5303 100644 --- a/python/cudf/cudf/io/parquet.py +++ b/python/cudf/cudf/io/parquet.py @@ -798,7 +798,6 @@ def _get_partitioned( filename=None, fs=None, preserve_index=False, - return_metadata=False, **kwargs, ): fs = ioutils._ensure_filesystem(fs, root_path, **kwargs) @@ -827,10 +826,9 @@ def _get_partitioned( filename = filename or uuid4().hex + ".parquet" full_path = fs.sep.join([prefix, filename]) full_paths.append(full_path) - if return_metadata: - metadata_file_paths.append(fs.sep.join([subdir, filename])) + metadata_file_paths.append(fs.sep.join([subdir, filename])) - return full_paths, grouped_df, part_offsets, filename + return full_paths, metadata_file_paths, grouped_df, part_offsets, filename class ParquetWriter: @@ -860,7 +858,11 @@ def __init__( self.filename = None if partition_cols is None: self._chunked_writers.append( - (libparquet.ParquetWriter([path], **self.common_args), []) + [ + libparquet.ParquetWriter([path], **self.common_args), + [], + None, + ] ) def write_table(self, df): @@ -868,7 +870,13 @@ def write_table(self, df): self._chunked_writers[0][0].write_table(df) return - paths, grouped_df, offsets, self.filename = _get_partitioned( + ( + paths, + metadata_file_paths, + grouped_df, + offsets, + self.filename, + ) = _get_partitioned( df, self.path, self.partition_cols, @@ -887,12 +895,14 @@ def pairwise(iterable): yield (a, b - a) a = b - for path, part_info in zip(paths, pairwise(offsets)): + for path, part_info, meta_path in zip( + paths, pairwise(offsets), metadata_file_paths + ): if path in self.path_cw_map: # path is a currently open file cw_idx = self.path_cw_map[path] existing_cw_batch[cw_idx][path] = part_info else: # path not currently handled by any chunked writer - new_cw_paths.append((path, part_info)) + new_cw_paths.append((path, part_info, meta_path)) # Write out the parts of grouped_df currently handled by existing cw's for batch_cw_path_list in existing_cw_batch.items(): @@ -907,21 +917,47 @@ def pairwise(iterable): cw.write_table(grouped_df, this_cw_part_info) # Create new cw for unhandled paths encountered in this write_table - new_paths = [path for path, _ in new_cw_paths] + new_paths = [path for path, _, _ in new_cw_paths] + meta_paths = [path for _, _, path in new_cw_paths] self._chunked_writers.append( - ( + [ libparquet.ParquetWriter(new_paths, **self.common_args), new_paths, - ) + meta_paths, + ] ) new_cw_idx = len(self._chunked_writers) - 1 self.path_cw_map.update({k: new_cw_idx for k in new_paths}) - part_info = [info for _, info in new_cw_paths] + part_info = [info for _, info, _ in new_cw_paths] self._chunked_writers[-1][0].write_table(grouped_df, part_info) - def close(self): - for cw, _ in self._chunked_writers: - cw.close() + def close(self, metadata_file_path=None): + return_metadata = bool(metadata_file_path) + if self.partition_cols is not None: + if isinstance(metadata_file_path, str): + warnings.warn( + "metadata_file_path is automatically determined for " + "partitioned writing. The passed metadata_file_path will " + "be ignored" + ) + else: + if return_metadata: + self._chunked_writers[0][2] = metadata_file_path + + metadata = [] + for cw, _, meta_path in self._chunked_writers: + metadata.append( + cw.close( + metadata_file_path=meta_path if return_metadata else None + ) + ) + + if return_metadata: + return ( + merge_parquet_filemetadata(metadata) + if len(metadata) > 1 + else metadata[0] + ) def __del__(self): self.close() From 48fb1117f10223a893876a514c9c6f4fa51a23e8 Mon Sep 17 00:00:00 2001 From: Devavret Makkar Date: Wed, 5 Jan 2022 18:39:40 +0530 Subject: [PATCH 56/71] Skip opening file which is not required for remote --- python/cudf/cudf/io/parquet.py | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/python/cudf/cudf/io/parquet.py b/python/cudf/cudf/io/parquet.py index 6c08550e8fc..4ad89165803 100644 --- a/python/cudf/cudf/io/parquet.py +++ b/python/cudf/cudf/io/parquet.py @@ -58,9 +58,7 @@ def _write_parquet( } if all([ioutils.is_fsspec_open_file(buf) for buf in paths_or_bufs]): with ExitStack() as stack: - fsspec_objs = [ - stack.enter_context(open(file)) for file in paths_or_bufs - ] + fsspec_objs = [stack.enter_context(file) for file in paths_or_bufs] file_objs = [ ioutils.get_IOBase_writer(file_obj) for file_obj in fsspec_objs ] From 0513d544ab0742a6ed9cb48dbb1e49296dc2814c Mon Sep 17 00:00:00 2001 From: Devavret Makkar Date: Fri, 7 Jan 2022 02:03:08 +0530 Subject: [PATCH 57/71] review changes --- python/cudf/cudf/_lib/io/utils.pyx | 4 ++++ python/cudf/cudf/_lib/parquet.pyx | 1 + 2 files changed, 5 insertions(+) diff --git a/python/cudf/cudf/_lib/io/utils.pyx b/python/cudf/cudf/_lib/io/utils.pyx index f3aa0cfdbc9..8e345bf969b 100644 --- a/python/cudf/cudf/_lib/io/utils.pyx +++ b/python/cudf/cudf/_lib/io/utils.pyx @@ -85,11 +85,13 @@ cdef sink_info make_sinks_info( cdef vector[data_sink *] data_sinks cdef vector[string] paths if isinstance(src[0], io.StringIO): + data_sinks.reserve(len(src)) for s in src: sink.push_back(unique_ptr[data_sink](new iobase_data_sink(s))) data_sinks.push_back(sink.back().get()) return sink_info(data_sinks) elif isinstance(src[0], io.TextIOBase): + data_sinks.reserve(len(src)) for s in src: # Files opened in text mode expect writes to be str rather than # bytes, which requires conversion from utf-8. If the underlying @@ -103,11 +105,13 @@ cdef sink_info make_sinks_info( data_sinks.push_back(sink.back().get()) return sink_info(data_sinks) elif isinstance(src[0], io.IOBase): + data_sinks.reserve(len(src)) for s in src: sink.push_back(unique_ptr[data_sink](new iobase_data_sink(s))) data_sinks.push_back(sink.back().get()) return sink_info(data_sinks) elif isinstance(src[0], (basestring, os.PathLike)): + paths.reserve(len(src)) for s in src: paths.push_back( os.path.expanduser(s).encode()) return sink_info(move(paths)) diff --git a/python/cudf/cudf/_lib/parquet.pyx b/python/cudf/cudf/_lib/parquet.pyx index ef3c2dccb00..36099b03ef6 100644 --- a/python/cudf/cudf/_lib/parquet.pyx +++ b/python/cudf/cudf/_lib/parquet.pyx @@ -366,6 +366,7 @@ cpdef write_parquet( .build() ) if partitions_info is not None: + partitions.reserve(len(partitions_info)) for part in partitions_info: partitions.push_back( cudf_io_types.partition_info(part[0], part[1]) From 5b00973dc415a99470a54e3ad5121aaca5767166 Mon Sep 17 00:00:00 2001 From: Devavret Makkar Date: Sat, 8 Jan 2022 04:40:20 +0530 Subject: [PATCH 58/71] Review changes --- python/cudf/cudf/io/parquet.py | 17 +++++++++-------- python/cudf/cudf/utils/ioutils.py | 18 ++++++++++++++++-- 2 files changed, 25 insertions(+), 10 deletions(-) diff --git a/python/cudf/cudf/io/parquet.py b/python/cudf/cudf/io/parquet.py index 4ad89165803..ca03e40e2a6 100644 --- a/python/cudf/cudf/io/parquet.py +++ b/python/cudf/cudf/io/parquet.py @@ -704,17 +704,19 @@ def to_parquet( if engine == "cudf": # Ensure that no columns dtype is 'category' for col in df.columns: - if df[col].dtype.name == "category": - raise ValueError( - "'category' column dtypes are currently not " - + "supported by the gpu accelerated parquet writer" - ) + if partition_cols is None or col not in partition_cols: + if df[col].dtype.name == "category": + raise ValueError( + "'category' column dtypes are currently not " + + "supported by the gpu accelerated parquet writer" + ) if partition_cols: if metadata_file_path is not None: warnings.warn( "metadata_file_path will be ignored/overwritten when " - "partition_cols are provided" + "partition_cols are provided. To request returning the " + "metadata binary blob, pass `return_metadata=True`" ) kwargs.update( { @@ -725,7 +727,7 @@ def to_parquet( "row_group_size_rows": row_group_size_rows, } ) - write_to_dataset( + return write_to_dataset( df, filename=partition_file_name, partition_cols=partition_cols, @@ -733,7 +735,6 @@ def to_parquet( preserve_index=index, **kwargs, ) - return if partition_offsets: kwargs["partitions_info"] = [ diff --git a/python/cudf/cudf/utils/ioutils.py b/python/cudf/cudf/utils/ioutils.py index c7ec539c6a6..b881f9372bc 100644 --- a/python/cudf/cudf/utils/ioutils.py +++ b/python/cudf/cudf/utils/ioutils.py @@ -198,9 +198,10 @@ Parameters ---------- -path : str +path : str or list of str File path or Root Directory path. Will be used as Root Directory path - while writing a partitioned dataset. + while writing a partitioned dataset. Use list of str with partition_offsets + to write parts of the dataframe to different files. compression : {'snappy', None}, default 'snappy' Name of the compression to use. Use ``None`` for no compression. index : bool, default None @@ -218,6 +219,16 @@ will be written to different directories, but all files will have this name. If nothing is specified, a random uuid4 hex string will be used for each file. +partition_offsets : list, optional, default None + Offsets to partition the dataframe by. Should be used when path is list + of str. Should be a list of integers of size ``len(path) + 1`` +statistics : {'ROWGROUP', 'PAGE', 'NONE'}, default 'ROWGROUP' + Level at which column statistics should be included in file. +metadata_file_path : str, optional, default None + If specified, this function will return a binary blob containing the footer + metadata of the written parquet file. The returned blob will have the + ``chunk.file_path`` field set to the ``metadata_file_path`` for each chunk. + When using with ``partition_offsets``, should be same size as ``len(path)`` int96_timestamps : bool, default False If ``True``, write timestamps in int96 format. This will convert timestamps from timestamp[ns], timestamp[ms], timestamp[s], and @@ -230,6 +241,9 @@ row_group_size_rows: integer or None, default None Maximum number of rows of each stripe of the output. If None, 1000000 will be used. +**kwargs + To request metadata binary blob when using with ``partition_cols``, Pass + ``return_metadata=True`` instead of specifying ``metadata_file_path`` See Also From 9d61a77d7cc73c277897bb3b7619c5d3ac98b56e Mon Sep 17 00:00:00 2001 From: Devavret Makkar Date: Mon, 10 Jan 2022 18:38:52 +0530 Subject: [PATCH 59/71] Docs, remove custom fs support --- python/cudf/cudf/io/parquet.py | 30 +++++++++++++++++++++++++++--- 1 file changed, 27 insertions(+), 3 deletions(-) diff --git a/python/cudf/cudf/io/parquet.py b/python/cudf/cudf/io/parquet.py index 868f7b8ae78..ca0629d71e9 100644 --- a/python/cudf/cudf/io/parquet.py +++ b/python/cudf/cudf/io/parquet.py @@ -838,8 +838,27 @@ def __init__( compression=None, statistics="ROWGROUP", partition_cols=None, - fs=None, ) -> None: + """ + Write a parquet file or dataset incrementally + + Parameters + ---------- + path : str + File path or Root Directory path. Will be used as Root Directory + path while writing a partitioned dataset. + index : bool, default None + If ``True``, include the dataframe’s index(es) in the file output. + If ``False``, they will not be written to the file. If ``None``, + index(es) other than RangeIndex will be saved as columns. + compression : {'snappy', None}, default 'snappy' + Name of the compression to use. Use ``None`` for no compression. + statistics : {'ROWGROUP', 'PAGE', 'NONE'}, default 'ROWGROUP' + Level at which column statistics should be included in file. + partition_cols : list, optional, default None + Column names by which to partition the dataset + Columns are partitioned in the order they are given + """ self.path = path self.common_args = { "index": index, @@ -853,7 +872,6 @@ def __init__( # Map of partition_col values to their libparquet.ParquetWriter's index # in self._chunked_writers for reverse lookup self.path_cw_map = {} - self.fs = fs self.filename = None if partition_cols is None: self._chunked_writers.append( @@ -865,6 +883,9 @@ def __init__( ) def write_table(self, df): + """ + Write a dataframe to the file/dataset + """ if self.partition_cols is None: self._chunked_writers[0][0].write_table(df) return @@ -880,7 +901,6 @@ def write_table(self, df): self.path, self.partition_cols, preserve_index=self.common_args["index"], - fs=self.fs, filename=self.filename, ) @@ -931,6 +951,10 @@ def pairwise(iterable): self._chunked_writers[-1][0].write_table(grouped_df, part_info) def close(self, metadata_file_path=None): + """ + Close all open files and optionally return footer metadata as a binary + blob + """ return_metadata = bool(metadata_file_path) if self.partition_cols is not None: if isinstance(metadata_file_path, str): From 912301fc4cc08c4b3acfee026cfbe0802b004f44 Mon Sep 17 00:00:00 2001 From: Devavret Makkar Date: Mon, 10 Jan 2022 20:49:36 +0530 Subject: [PATCH 60/71] remove redundant partitioning logic in write_to_dataset --- python/cudf/cudf/io/parquet.py | 41 +++++++++++++--------------------- 1 file changed, 15 insertions(+), 26 deletions(-) diff --git a/python/cudf/cudf/io/parquet.py b/python/cudf/cudf/io/parquet.py index ca0629d71e9..3d80d77539d 100644 --- a/python/cudf/cudf/io/parquet.py +++ b/python/cudf/cudf/io/parquet.py @@ -126,32 +126,21 @@ def write_to_dataset( if partition_cols is not None and len(partition_cols) > 0: - data_cols = df.columns.drop(partition_cols) - if len(data_cols) == 0: - raise ValueError("No data left to save outside partition columns") - - part_names, part_offsets, _, grouped_df = df.groupby( - partition_cols - )._grouped() - if not preserve_index: - grouped_df.reset_index(drop=True, inplace=True) - grouped_df.drop(columns=partition_cols, inplace=True) - # Copy the entire keys df in one operation rather than using iloc - part_names = part_names.to_pandas().to_frame(index=False) - - full_paths = [] - metadata_file_paths = [] - for keys in part_names.itertuples(index=False): - subdir = fs.sep.join( - [f"{name}={val}" for name, val in zip(partition_cols, keys)] - ) - prefix = fs.sep.join([root_path, subdir]) - fs.mkdirs(prefix, exist_ok=True) - filename = filename or uuid4().hex + ".parquet" - full_path = fs.sep.join([prefix, filename]) - full_paths.append(full_path) - if return_metadata: - metadata_file_paths.append(fs.sep.join([subdir, filename])) + ( + full_paths, + metadata_file_paths, + grouped_df, + part_offsets, + _, + ) = _get_partitioned( + df, + root_path, + partition_cols, + filename, + fs, + preserve_index, + **kwargs, + ) if return_metadata: kwargs["metadata_file_path"] = metadata_file_paths From bb30a09ff02435ea11da3150e4508cfe5943f489 Mon Sep 17 00:00:00 2001 From: Devavret Makkar Date: Mon, 10 Jan 2022 20:51:17 +0530 Subject: [PATCH 61/71] fix cython style --- python/cudf/cudf/_lib/parquet.pyx | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/python/cudf/cudf/_lib/parquet.pyx b/python/cudf/cudf/_lib/parquet.pyx index 3ec34d41c42..100585f1977 100644 --- a/python/cudf/cudf/_lib/parquet.pyx +++ b/python/cudf/cudf/_lib/parquet.pyx @@ -515,7 +515,9 @@ cdef class ParquetWriter: table[name]._column, self.tbl_meta.get().column_metadata[i] ) - index = False if isinstance(table._index, cudf.RangeIndex) else self.index + index = ( + False if isinstance(table._index, cudf.RangeIndex) else self.index + ) pandas_metadata = generate_pandas_metadata(table, index) cdef map[string, string] tmp_user_data tmp_user_data[str.encode("pandas")] = str.encode(pandas_metadata) From 18c352459ba3a6324e326ac4cdb683564abff279 Mon Sep 17 00:00:00 2001 From: Devavret Makkar Date: Tue, 11 Jan 2022 03:47:36 +0530 Subject: [PATCH 62/71] Add pytest --- python/cudf/cudf/tests/test_parquet.py | 25 +++++++++++++++++++++++++ 1 file changed, 25 insertions(+) diff --git a/python/cudf/cudf/tests/test_parquet.py b/python/cudf/cudf/tests/test_parquet.py index 9a66de8a3a6..7599e24754b 100644 --- a/python/cudf/cudf/tests/test_parquet.py +++ b/python/cudf/cudf/tests/test_parquet.py @@ -1627,6 +1627,31 @@ def test_parquet_partitioned(tmpdir_factory, cols, filename): assert fn == filename +def test_parquet_writer_chunked_partitioned(tmpdir_factory): + pdf_dir = str(tmpdir_factory.mktemp("pdf_dir")) + gdf_dir = str(tmpdir_factory.mktemp("gdf_dir")) + + df1 = cudf.DataFrame({"a": [1, 1, 2, 2, 1], "b": [9, 8, 7, 6, 5]}) + df2 = cudf.DataFrame({"a": [1, 3, 3, 1, 3], "b": [4, 3, 2, 1, 0]}) + + cw = ParquetWriter(gdf_dir, partition_cols=["a"], index=False) + cw.write_table(df1) + cw.write_table(df2) + cw.close(metadata_file_path=True) + + pdf = cudf.concat([df1, df2]).to_pandas() + pdf.to_parquet(pdf_dir, index=False, partition_cols=["a"]) + + # Read back with pandas to compare + expect_pd = pd.read_parquet(pdf_dir) + got_pd = pd.read_parquet(gdf_dir) + assert_eq(expect_pd, got_pd) + + # Check that cudf and pd return the same read + got_cudf = cudf.read_parquet(gdf_dir) + assert_eq(got_pd, got_cudf) + + @pytest.mark.parametrize("cols", [None, ["b"]]) def test_parquet_write_to_dataset(tmpdir_factory, cols): dir1 = tmpdir_factory.mktemp("dir1") From cd698edfeb7cc1b54bfd3db7b4d35ab0c1dc2ca9 Mon Sep 17 00:00:00 2001 From: Devavret Makkar Date: Tue, 11 Jan 2022 16:56:23 +0530 Subject: [PATCH 63/71] mypy fixes --- python/cudf/cudf/io/parquet.py | 18 +++++++++++------- 1 file changed, 11 insertions(+), 7 deletions(-) diff --git a/python/cudf/cudf/io/parquet.py b/python/cudf/cudf/io/parquet.py index 3d80d77539d..f2c95acec8c 100644 --- a/python/cudf/cudf/io/parquet.py +++ b/python/cudf/cudf/io/parquet.py @@ -5,6 +5,7 @@ import warnings from collections import defaultdict from contextlib import ExitStack +from typing import Dict, List, Tuple from uuid import uuid4 import fsspec @@ -857,18 +858,20 @@ def __init__( self.partition_cols = partition_cols # Collection of `libparquet.ParquetWriter`s, and the corresponding # partition_col values they're responsible for - self._chunked_writers = [] + self._chunked_writers: List[ + Tuple[libparquet.ParquetWriter, List[str], List[str]] + ] = [] # Map of partition_col values to their libparquet.ParquetWriter's index # in self._chunked_writers for reverse lookup - self.path_cw_map = {} + self.path_cw_map: Dict[str, int] = {} self.filename = None if partition_cols is None: self._chunked_writers.append( - [ + ( libparquet.ParquetWriter([path], **self.common_args), [], None, - ] + ) ) def write_table(self, df): @@ -928,11 +931,11 @@ def pairwise(iterable): new_paths = [path for path, _, _ in new_cw_paths] meta_paths = [path for _, _, path in new_cw_paths] self._chunked_writers.append( - [ + ( libparquet.ParquetWriter(new_paths, **self.common_args), new_paths, meta_paths, - ] + ) ) new_cw_idx = len(self._chunked_writers) - 1 self.path_cw_map.update({k: new_cw_idx for k in new_paths}) @@ -954,7 +957,8 @@ def close(self, metadata_file_path=None): ) else: if return_metadata: - self._chunked_writers[0][2] = metadata_file_path + cw, paths, _ = self._chunked_writers.pop() + self._chunked_writers.append((cw, paths, metadata_file_path)) metadata = [] for cw, _, meta_path in self._chunked_writers: From b717373b44cab4f5da2fcc2d67a03dc6be941add Mon Sep 17 00:00:00 2001 From: Devavret Makkar Date: Tue, 11 Jan 2022 17:09:04 +0530 Subject: [PATCH 64/71] More mypy fix --- python/cudf/cudf/io/parquet.py | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/python/cudf/cudf/io/parquet.py b/python/cudf/cudf/io/parquet.py index f2c95acec8c..ad6f8311f08 100644 --- a/python/cudf/cudf/io/parquet.py +++ b/python/cudf/cudf/io/parquet.py @@ -859,7 +859,7 @@ def __init__( # Collection of `libparquet.ParquetWriter`s, and the corresponding # partition_col values they're responsible for self._chunked_writers: List[ - Tuple[libparquet.ParquetWriter, List[str], List[str]] + Tuple[libparquet.ParquetWriter, List[str], str] ] = [] # Map of partition_col values to their libparquet.ParquetWriter's index # in self._chunked_writers for reverse lookup @@ -867,11 +867,7 @@ def __init__( self.filename = None if partition_cols is None: self._chunked_writers.append( - ( - libparquet.ParquetWriter([path], **self.common_args), - [], - None, - ) + (libparquet.ParquetWriter([path], **self.common_args), [], "") ) def write_table(self, df): From 7278f4f8c9d4b272515bce6888fd2a18abfdc18d Mon Sep 17 00:00:00 2001 From: Devavret Makkar Date: Wed, 12 Jan 2022 03:44:28 +0530 Subject: [PATCH 65/71] Review fixes --- python/cudf/cudf/_lib/cpp/io/parquet.pxd | 4 +- python/cudf/cudf/_lib/parquet.pyx | 5 ++ python/cudf/cudf/io/parquet.py | 74 +++++++++--------------- python/cudf/cudf/tests/test_parquet.py | 10 +++- 4 files changed, 40 insertions(+), 53 deletions(-) diff --git a/python/cudf/cudf/_lib/cpp/io/parquet.pxd b/python/cudf/cudf/_lib/cpp/io/parquet.pxd index 93780412f37..d02fffe9c0d 100644 --- a/python/cudf/cudf/_lib/cpp/io/parquet.pxd +++ b/python/cudf/cudf/_lib/cpp/io/parquet.pxd @@ -208,8 +208,8 @@ cdef extern from "cudf/io/parquet.hpp" namespace "cudf::io" nogil: cudf_table_view.table_view table_, ) except+ parquet_chunked_writer& write( - cudf_table_view.table_view table_, - vector[cudf_io_types.partition_info] partitions, + const cudf_table_view.table_view& table_, + const vector[cudf_io_types.partition_info]& partitions, ) except+ unique_ptr[vector[uint8_t]] close( vector[string] column_chunks_file_paths, diff --git a/python/cudf/cudf/_lib/parquet.pyx b/python/cudf/cudf/_lib/parquet.pyx index 100585f1977..a76b773f4b0 100644 --- a/python/cudf/cudf/_lib/parquet.pyx +++ b/python/cudf/cudf/_lib/parquet.pyx @@ -418,6 +418,11 @@ cdef class ParquetWriter: def __cinit__(self, object filepaths_or_buffers, object index=None, object compression=None, str statistics="ROWGROUP"): + filepaths_or_buffers = ( + list(filepaths_or_buffers) + if is_list_like(filepaths_or_buffers) + else [filepaths_or_buffers] + ) self.sink = make_sinks_info(filepaths_or_buffers, self._data_sink) self.stat_freq = _get_stat_freq(statistics) self.comp_type = _get_comp_type(compression) diff --git a/python/cudf/cudf/io/parquet.py b/python/cudf/cudf/io/parquet.py index ad6f8311f08..519f5089c8e 100644 --- a/python/cudf/cudf/io/parquet.py +++ b/python/cudf/cudf/io/parquet.py @@ -154,7 +154,7 @@ def write_to_dataset( ) else: - filename = filename or uuid4().hex + ".parquet" + filename = filename or (uuid4().hex + ".parquet") full_path = fs.sep.join([root_path, filename]) if return_metadata: kwargs["metadata_file_path"] = filename @@ -812,7 +812,7 @@ def _get_partitioned( ) prefix = fs.sep.join([root_path, subdir]) fs.mkdirs(prefix, exist_ok=True) - filename = filename or uuid4().hex + ".parquet" + filename = filename or (uuid4().hex + ".parquet") full_path = fs.sep.join([prefix, filename]) full_paths.append(full_path) metadata_file_paths.append(fs.sep.join([subdir, filename])) @@ -820,14 +820,17 @@ def _get_partitioned( return full_paths, metadata_file_paths, grouped_df, part_offsets, filename -class ParquetWriter: +ParquetWriter = libparquet.ParquetWriter + + +class ParquetDatasetWriter: def __init__( self, path, + partition_cols, index=None, compression=None, statistics="ROWGROUP", - partition_cols=None, ) -> None: """ Write a parquet file or dataset incrementally @@ -837,6 +840,9 @@ def __init__( path : str File path or Root Directory path. Will be used as Root Directory path while writing a partitioned dataset. + partition_cols : list + Column names by which to partition the dataset + Columns are partitioned in the order they are given index : bool, default None If ``True``, include the dataframe’s index(es) in the file output. If ``False``, they will not be written to the file. If ``None``, @@ -845,9 +851,6 @@ def __init__( Name of the compression to use. Use ``None`` for no compression. statistics : {'ROWGROUP', 'PAGE', 'NONE'}, default 'ROWGROUP' Level at which column statistics should be included in file. - partition_cols : list, optional, default None - Column names by which to partition the dataset - Columns are partitioned in the order they are given """ self.path = path self.common_args = { @@ -856,28 +859,18 @@ def __init__( "statistics": statistics, } self.partition_cols = partition_cols - # Collection of `libparquet.ParquetWriter`s, and the corresponding + # Collection of `ParquetWriter`s, and the corresponding # partition_col values they're responsible for - self._chunked_writers: List[ - Tuple[libparquet.ParquetWriter, List[str], str] - ] = [] - # Map of partition_col values to their libparquet.ParquetWriter's index + self._chunked_writers: List[Tuple[ParquetWriter, List[str], str]] = [] + # Map of partition_col values to their ParquetWriter's index # in self._chunked_writers for reverse lookup self.path_cw_map: Dict[str, int] = {} self.filename = None - if partition_cols is None: - self._chunked_writers.append( - (libparquet.ParquetWriter([path], **self.common_args), [], "") - ) def write_table(self, df): """ Write a dataframe to the file/dataset """ - if self.partition_cols is None: - self._chunked_writers[0][0].write_table(df) - return - ( paths, metadata_file_paths, @@ -896,6 +889,9 @@ def write_table(self, df): new_cw_paths = [] def pairwise(iterable): + """ + Generates a pair of `(it[i], it[i + 1] - it[i])` from iterable `it` + """ it = iter(iterable) a = next(it, None) for b in it: @@ -912,57 +908,39 @@ def pairwise(iterable): new_cw_paths.append((path, part_info, meta_path)) # Write out the parts of grouped_df currently handled by existing cw's - for batch_cw_path_list in existing_cw_batch.items(): - cw_idx = batch_cw_path_list[0] + for cw_idx, path_to_part_info_map in existing_cw_batch.items(): cw = self._chunked_writers[cw_idx][0] # match found paths with this cw's paths and nullify partition info # for partition_col values not in this batch this_cw_part_info = [ - batch_cw_path_list[1].get(path, (0, 0)) + path_to_part_info_map.get(path, (0, 0)) for path in self._chunked_writers[cw_idx][1] ] cw.write_table(grouped_df, this_cw_part_info) # Create new cw for unhandled paths encountered in this write_table - new_paths = [path for path, _, _ in new_cw_paths] - meta_paths = [path for _, _, path in new_cw_paths] + new_paths, part_info, meta_paths = zip(*new_cw_paths) self._chunked_writers.append( ( - libparquet.ParquetWriter(new_paths, **self.common_args), + ParquetWriter(new_paths, **self.common_args), new_paths, meta_paths, ) ) new_cw_idx = len(self._chunked_writers) - 1 self.path_cw_map.update({k: new_cw_idx for k in new_paths}) - part_info = [info for _, info, _ in new_cw_paths] self._chunked_writers[-1][0].write_table(grouped_df, part_info) - def close(self, metadata_file_path=None): + def close(self, return_metadata=False): """ Close all open files and optionally return footer metadata as a binary blob """ - return_metadata = bool(metadata_file_path) - if self.partition_cols is not None: - if isinstance(metadata_file_path, str): - warnings.warn( - "metadata_file_path is automatically determined for " - "partitioned writing. The passed metadata_file_path will " - "be ignored" - ) - else: - if return_metadata: - cw, paths, _ = self._chunked_writers.pop() - self._chunked_writers.append((cw, paths, metadata_file_path)) - - metadata = [] - for cw, _, meta_path in self._chunked_writers: - metadata.append( - cw.close( - metadata_file_path=meta_path if return_metadata else None - ) - ) + + metadata = [ + cw.close(metadata_file_path=meta_path if return_metadata else None) + for cw, _, meta_path in self._chunked_writers + ] if return_metadata: return ( diff --git a/python/cudf/cudf/tests/test_parquet.py b/python/cudf/cudf/tests/test_parquet.py index 7599e24754b..c2b9d62814d 100644 --- a/python/cudf/cudf/tests/test_parquet.py +++ b/python/cudf/cudf/tests/test_parquet.py @@ -18,7 +18,11 @@ from pyarrow import fs as pa_fs, parquet as pq import cudf -from cudf.io.parquet import ParquetWriter, merge_parquet_filemetadata +from cudf.io.parquet import ( + ParquetWriter, + ParquetDatasetWriter, + merge_parquet_filemetadata, +) from cudf.testing import dataset_generator as dg from cudf.testing._utils import ( TIMEDELTA_TYPES, @@ -1634,10 +1638,10 @@ def test_parquet_writer_chunked_partitioned(tmpdir_factory): df1 = cudf.DataFrame({"a": [1, 1, 2, 2, 1], "b": [9, 8, 7, 6, 5]}) df2 = cudf.DataFrame({"a": [1, 3, 3, 1, 3], "b": [4, 3, 2, 1, 0]}) - cw = ParquetWriter(gdf_dir, partition_cols=["a"], index=False) + cw = ParquetDatasetWriter(gdf_dir, partition_cols=["a"], index=False) cw.write_table(df1) cw.write_table(df2) - cw.close(metadata_file_path=True) + cw.close() pdf = cudf.concat([df1, df2]).to_pandas() pdf.to_parquet(pdf_dir, index=False, partition_cols=["a"]) From 4b0efe5da5426016e39583b30d93d54470e32556 Mon Sep 17 00:00:00 2001 From: Devavret Makkar Date: Wed, 12 Jan 2022 03:48:56 +0530 Subject: [PATCH 66/71] 1 more review change --- python/cudf/cudf/io/parquet.py | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/python/cudf/cudf/io/parquet.py b/python/cudf/cudf/io/parquet.py index 519f5089c8e..ac7aa7a74fa 100644 --- a/python/cudf/cudf/io/parquet.py +++ b/python/cudf/cudf/io/parquet.py @@ -791,8 +791,7 @@ def _get_partitioned( ): fs = ioutils._ensure_filesystem(fs, root_path, **kwargs) fs.mkdirs(root_path, exist_ok=True) - data_cols = df.columns.drop(partition_cols) - if len(data_cols) == 0: + if not (set(df._data) - set(partition_cols)): raise ValueError("No data left to save outside partition columns") part_names, part_offsets, _, grouped_df = df.groupby( From ef69e62086f95d2fd25b2ef7b097562daf7c66bc Mon Sep 17 00:00:00 2001 From: Devavret Makkar Date: Wed, 12 Jan 2022 05:32:34 +0530 Subject: [PATCH 67/71] Add returned meta test --- python/cudf/cudf/tests/test_parquet.py | 22 +++++++++++++++++++--- 1 file changed, 19 insertions(+), 3 deletions(-) diff --git a/python/cudf/cudf/tests/test_parquet.py b/python/cudf/cudf/tests/test_parquet.py index c2b9d62814d..47d76e75aa4 100644 --- a/python/cudf/cudf/tests/test_parquet.py +++ b/python/cudf/cudf/tests/test_parquet.py @@ -1631,7 +1631,8 @@ def test_parquet_partitioned(tmpdir_factory, cols, filename): assert fn == filename -def test_parquet_writer_chunked_partitioned(tmpdir_factory): +@pytest.mark.parametrize("return_meta", [True, False]) +def test_parquet_writer_chunked_partitioned(tmpdir_factory, return_meta): pdf_dir = str(tmpdir_factory.mktemp("pdf_dir")) gdf_dir = str(tmpdir_factory.mktemp("gdf_dir")) @@ -1641,11 +1642,26 @@ def test_parquet_writer_chunked_partitioned(tmpdir_factory): cw = ParquetDatasetWriter(gdf_dir, partition_cols=["a"], index=False) cw.write_table(df1) cw.write_table(df2) - cw.close() - + meta_byte_array = cw.close(return_metadata=return_meta) pdf = cudf.concat([df1, df2]).to_pandas() pdf.to_parquet(pdf_dir, index=False, partition_cols=["a"]) + if return_meta: + fmd = pq.ParquetFile(BytesIO(meta_byte_array)).metadata + assert fmd.num_rows == len(pdf) + assert fmd.num_row_groups == 4 + files = { + os.path.join(directory, files[0]) + for directory, _, files in os.walk(gdf_dir) + if files + } + meta_files = { + os.path.join(gdf_dir, fmd.row_group(i).column(c).file_path) + for i in range(fmd.num_row_groups) + for c in range(fmd.row_group(i).num_columns) + } + assert files == meta_files + # Read back with pandas to compare expect_pd = pd.read_parquet(pdf_dir) got_pd = pd.read_parquet(gdf_dir) From fc4a6df600fc042dc95b2923e06432cf5c9eb623 Mon Sep 17 00:00:00 2001 From: Devavret Makkar Date: Wed, 12 Jan 2022 15:23:22 +0530 Subject: [PATCH 68/71] mypy fix --- python/cudf/cudf/io/parquet.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/python/cudf/cudf/io/parquet.py b/python/cudf/cudf/io/parquet.py index ac7aa7a74fa..ebe45c6bc04 100644 --- a/python/cudf/cudf/io/parquet.py +++ b/python/cudf/cudf/io/parquet.py @@ -860,7 +860,9 @@ def __init__( self.partition_cols = partition_cols # Collection of `ParquetWriter`s, and the corresponding # partition_col values they're responsible for - self._chunked_writers: List[Tuple[ParquetWriter, List[str], str]] = [] + self._chunked_writers: List[ + Tuple[libparquet.ParquetWriter, List[str], str] + ] = [] # Map of partition_col values to their ParquetWriter's index # in self._chunked_writers for reverse lookup self.path_cw_map: Dict[str, int] = {} From e1d608abdfe9b78f4f2cf32f1faef184b99dcaad Mon Sep 17 00:00:00 2001 From: Devavret Makkar Date: Wed, 12 Jan 2022 15:51:38 +0530 Subject: [PATCH 69/71] Another style fix --- python/cudf/cudf/tests/test_parquet.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/cudf/cudf/tests/test_parquet.py b/python/cudf/cudf/tests/test_parquet.py index 47d76e75aa4..076a0a817b8 100644 --- a/python/cudf/cudf/tests/test_parquet.py +++ b/python/cudf/cudf/tests/test_parquet.py @@ -19,8 +19,8 @@ import cudf from cudf.io.parquet import ( - ParquetWriter, ParquetDatasetWriter, + ParquetWriter, merge_parquet_filemetadata, ) from cudf.testing import dataset_generator as dg From 70612e6139ee4c70bfb8399dafea6cdd031f41c3 Mon Sep 17 00:00:00 2001 From: Devavret Makkar Date: Thu, 13 Jan 2022 02:09:30 +0530 Subject: [PATCH 70/71] Remove destructor in favour of contextlib --- python/cudf/cudf/_lib/parquet.pyx | 5 +++- python/cudf/cudf/io/parquet.py | 5 +++- python/cudf/cudf/tests/test_parquet.py | 36 ++++++++++++++++++++++++++ 3 files changed, 44 insertions(+), 2 deletions(-) diff --git a/python/cudf/cudf/_lib/parquet.pyx b/python/cudf/cudf/_lib/parquet.pyx index a76b773f4b0..16873435e1d 100644 --- a/python/cudf/cudf/_lib/parquet.pyx +++ b/python/cudf/cudf/_lib/parquet.pyx @@ -484,7 +484,10 @@ cdef class ParquetWriter: return np.asarray(out_metadata_py) return None - def __dealloc__(self): + def __enter__(self): + return self + + def __exit__(self, *args): self.close() def _initialize_chunked_state(self, table, num_partitions=1): diff --git a/python/cudf/cudf/io/parquet.py b/python/cudf/cudf/io/parquet.py index ebe45c6bc04..bd4de908f49 100644 --- a/python/cudf/cudf/io/parquet.py +++ b/python/cudf/cudf/io/parquet.py @@ -950,7 +950,10 @@ def close(self, return_metadata=False): else metadata[0] ) - def __del__(self): + def __enter__(self): + return self + + def __exit__(self, *args): self.close() diff --git a/python/cudf/cudf/tests/test_parquet.py b/python/cudf/cudf/tests/test_parquet.py index 076a0a817b8..016ed1229f1 100644 --- a/python/cudf/cudf/tests/test_parquet.py +++ b/python/cudf/cudf/tests/test_parquet.py @@ -1577,6 +1577,16 @@ def test_parquet_writer_gpu_chunked(tmpdir, simple_pdf, simple_gdf): assert_eq(pd.read_parquet(gdf_fname), pd.concat([simple_pdf, simple_pdf])) +def test_parquet_writer_gpu_chunked_context(tmpdir, simple_pdf, simple_gdf): + gdf_fname = tmpdir.join("gdf.parquet") + + with ParquetWriter(gdf_fname) as writer: + writer.write_table(simple_gdf) + writer.write_table(simple_gdf) + + assert_eq(pd.read_parquet(gdf_fname), pd.concat([simple_pdf, simple_pdf])) + + def test_parquet_write_bytes_io(simple_gdf): output = BytesIO() simple_gdf.to_parquet(output) @@ -1672,6 +1682,32 @@ def test_parquet_writer_chunked_partitioned(tmpdir_factory, return_meta): assert_eq(got_pd, got_cudf) +def test_parquet_writer_chunked_partitioned_context(tmpdir_factory): + pdf_dir = str(tmpdir_factory.mktemp("pdf_dir")) + gdf_dir = str(tmpdir_factory.mktemp("gdf_dir")) + + df1 = cudf.DataFrame({"a": [1, 1, 2, 2, 1], "b": [9, 8, 7, 6, 5]}) + df2 = cudf.DataFrame({"a": [1, 3, 3, 1, 3], "b": [4, 3, 2, 1, 0]}) + + with ParquetDatasetWriter( + gdf_dir, partition_cols=["a"], index=False + ) as cw: + cw.write_table(df1) + cw.write_table(df2) + + pdf = cudf.concat([df1, df2]).to_pandas() + pdf.to_parquet(pdf_dir, index=False, partition_cols=["a"]) + + # Read back with pandas to compare + expect_pd = pd.read_parquet(pdf_dir) + got_pd = pd.read_parquet(gdf_dir) + assert_eq(expect_pd, got_pd) + + # Check that cudf and pd return the same read + got_cudf = cudf.read_parquet(gdf_dir) + assert_eq(got_pd, got_cudf) + + @pytest.mark.parametrize("cols", [None, ["b"]]) def test_parquet_write_to_dataset(tmpdir_factory, cols): dir1 = tmpdir_factory.mktemp("dir1") From 6552fbee05d8a814d9cae9c1d3c417d78c98239c Mon Sep 17 00:00:00 2001 From: Devavret Makkar Date: Fri, 14 Jan 2022 02:41:28 +0530 Subject: [PATCH 71/71] Review changes - Replace part_info generator with numpy roll - Add examples to docs --- python/cudf/cudf/io/parquet.py | 65 ++++++++++++++++++++++++---------- 1 file changed, 46 insertions(+), 19 deletions(-) diff --git a/python/cudf/cudf/io/parquet.py b/python/cudf/cudf/io/parquet.py index bd4de908f49..9694d19e159 100644 --- a/python/cudf/cudf/io/parquet.py +++ b/python/cudf/cudf/io/parquet.py @@ -9,6 +9,7 @@ from uuid import uuid4 import fsspec +import numpy as np import pyarrow as pa from pyarrow import dataset as ds, parquet as pq @@ -154,7 +155,7 @@ def write_to_dataset( ) else: - filename = filename or (uuid4().hex + ".parquet") + filename = filename or _generate_filename() full_path = fs.sep.join([root_path, filename]) if return_metadata: kwargs["metadata_file_path"] = filename @@ -727,13 +728,12 @@ def to_parquet( ) if partition_offsets: - kwargs["partitions_info"] = [ - ( - partition_offsets[i], - partition_offsets[i + 1] - partition_offsets[i], + kwargs["partitions_info"] = list( + zip( + partition_offsets, + np.roll(partition_offsets, -1) - partition_offsets, ) - for i in range(0, len(partition_offsets) - 1) - ] + )[:-1] return _write_parquet( df, @@ -780,6 +780,10 @@ def merge_parquet_filemetadata(filemetadata_list): return libparquet.merge_filemetadata(filemetadata_list) +def _generate_filename(): + return uuid4().hex + ".parquet" + + def _get_partitioned( df, root_path, @@ -811,7 +815,7 @@ def _get_partitioned( ) prefix = fs.sep.join([root_path, subdir]) fs.mkdirs(prefix, exist_ok=True) - filename = filename or (uuid4().hex + ".parquet") + filename = filename or _generate_filename() full_path = fs.sep.join([prefix, filename]) full_paths.append(full_path) metadata_file_paths.append(fs.sep.join([subdir, filename])) @@ -850,6 +854,37 @@ def __init__( Name of the compression to use. Use ``None`` for no compression. statistics : {'ROWGROUP', 'PAGE', 'NONE'}, default 'ROWGROUP' Level at which column statistics should be included in file. + + + Examples + ________ + Using a context + + >>> df1 = cudf.DataFrame({"a": [1, 1, 2, 2, 1], "b": [9, 8, 7, 6, 5]}) + >>> df2 = cudf.DataFrame({"a": [1, 3, 3, 1, 3], "b": [4, 3, 2, 1, 0]}) + >>> with ParquetDatasetWriter("./dataset", partition_cols=["a"]) as cw: + ... cw.write_table(df1) + ... cw.write_table(df2) + + By manually calling ``close()`` + + >>> cw = ParquetDatasetWriter("./dataset", partition_cols=["a"]) + >>> cw.write_table(df1) + >>> cw.write_table(df2) + >>> cw.close() + + Both the methods will generate the same directory structure + + .. code-block:: bash + + dataset/ + a=1 + .parquet + a=2 + .parquet + a=3 + .parquet + """ self.path = path self.common_args = { @@ -889,18 +924,10 @@ def write_table(self, df): existing_cw_batch = defaultdict(dict) new_cw_paths = [] - def pairwise(iterable): - """ - Generates a pair of `(it[i], it[i + 1] - it[i])` from iterable `it` - """ - it = iter(iterable) - a = next(it, None) - for b in it: - yield (a, b - a) - a = b - for path, part_info, meta_path in zip( - paths, pairwise(offsets), metadata_file_paths + paths, + zip(offsets, np.roll(offsets, -1) - offsets), + metadata_file_paths, ): if path in self.path_cw_map: # path is a currently open file cw_idx = self.path_cw_map[path]