From d441f51ad0d6860e0d3c25837d8ecc55076a1be5 Mon Sep 17 00:00:00 2001 From: David Wendt <45795991+davidwendt@users.noreply.github.com> Date: Wed, 22 Feb 2023 09:46:25 -0500 Subject: [PATCH 01/60] Enable groupby std and variance aggregation types in libcudf Debug build (#12799) Re-enable groupby with `std` and `var` aggregations that were disabled for Debug builds due to a runtime issue. Retesting with nvcc 11.5, the error is no longer present so the code and the gtests have been re-enabled. Found while working on PR #12784 Authors: - David Wendt (https://github.com/davidwendt) Approvers: - Bradley Dice (https://github.com/bdice) - Mike Wilson (https://github.com/hyperbolic2346) - Vyas Ramasubramani (https://github.com/vyasr) URL: https://github.com/rapidsai/cudf/pull/12799 --- cpp/src/groupby/sort/group_std.cu | 8 +------- cpp/tests/groupby/std_tests.cpp | 28 ---------------------------- cpp/tests/groupby/var_tests.cpp | 28 ---------------------------- 3 files changed, 1 insertion(+), 63 deletions(-) diff --git a/cpp/src/groupby/sort/group_std.cu b/cpp/src/groupby/sort/group_std.cu index 87fd9f7e843..a3efc1f172a 100644 --- a/cpp/src/groupby/sort/group_std.cu +++ b/cpp/src/groupby/sort/group_std.cu @@ -1,5 +1,5 @@ /* - * Copyright (c) 2019-2022, NVIDIA CORPORATION. + * Copyright (c) 2019-2023, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -99,9 +99,6 @@ struct var_functor { rmm::cuda_stream_view stream, rmm::mr::device_memory_resource* mr) { -// Running this in debug build causes a runtime error: -// `reduce_by_key failed on 2nd step: invalid device function` -#if !defined(__CUDACC_DEBUG__) using ResultType = cudf::detail::target_type_t; std::unique_ptr result = make_numeric_column(data_type(type_to_id()), @@ -141,9 +138,6 @@ struct var_functor { }); return result; -#else - CUDF_FAIL("Groupby std/var supported in debug build"); -#endif } template diff --git a/cpp/tests/groupby/std_tests.cpp b/cpp/tests/groupby/std_tests.cpp index fa3afeb30f8..56ddce1554f 100644 --- a/cpp/tests/groupby/std_tests.cpp +++ b/cpp/tests/groupby/std_tests.cpp @@ -33,11 +33,7 @@ using supported_types = cudf::test::Types Date: Wed, 22 Feb 2023 09:42:46 -0800 Subject: [PATCH 02/60] Variable fragment sizes for Parquet writer (#12685) Fixes #12613 This PR adds the ability for columns to have different fragment sizes. This allows a large fragment size for narrow columns, but allows for finer grained fragments for very wide columns. This change should make wide columns fit (approximately) within page size constraints, and should help with compressors that rely on pages being under a certain threshold (i.e. Zstandard). Authors: - Ed Seidl (https://github.com/etseidl) - Vukasin Milovanovic (https://github.com/vuule) - Mike Wilson (https://github.com/hyperbolic2346) Approvers: - Vukasin Milovanovic (https://github.com/vuule) - Mike Wilson (https://github.com/hyperbolic2346) URL: https://github.com/rapidsai/cudf/pull/12685 --- cpp/include/cudf/io/parquet.hpp | 4 +- cpp/src/io/parquet/page_enc.cu | 251 ++++++++++-------- cpp/src/io/parquet/parquet_gpu.hpp | 43 ++- cpp/src/io/parquet/writer_impl.cu | 265 ++++++++++++++----- cpp/src/io/parquet/writer_impl.hpp | 42 +-- cpp/tests/io/parquet_chunked_reader_test.cpp | 3 +- cpp/tests/io/parquet_test.cpp | 46 +++- 7 files changed, 449 insertions(+), 205 deletions(-) diff --git a/cpp/include/cudf/io/parquet.hpp b/cpp/include/cudf/io/parquet.hpp index f4fb4d91f58..92b69deb671 100644 --- a/cpp/include/cudf/io/parquet.hpp +++ b/cpp/include/cudf/io/parquet.hpp @@ -494,7 +494,7 @@ class parquet_writer_options { // Maximum size of column chunk dictionary (in bytes) size_t _max_dictionary_size = default_max_dictionary_size; // Maximum number of rows in a page fragment - size_type _max_page_fragment_size = default_max_page_fragment_size; + std::optional _max_page_fragment_size; /** * @brief Constructor from sink and table. @@ -1076,7 +1076,7 @@ class chunked_parquet_writer_options { // Maximum size of column chunk dictionary (in bytes) size_t _max_dictionary_size = default_max_dictionary_size; // Maximum number of rows in a page fragment - size_type _max_page_fragment_size = default_max_page_fragment_size; + std::optional _max_page_fragment_size; /** * @brief Constructor from sink. diff --git a/cpp/src/io/parquet/page_enc.cu b/cpp/src/io/parquet/page_enc.cu index 9f8f42702cd..5a12acec2a3 100644 --- a/cpp/src/io/parquet/page_enc.cu +++ b/cpp/src/io/parquet/page_enc.cu @@ -1,5 +1,5 @@ /* - * Copyright (c) 2019-2022, NVIDIA CORPORATION. + * Copyright (c) 2019-2023, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -126,129 +126,164 @@ constexpr size_t underflow_safe_subtract(size_t a, size_t b) return a - b; } +void __device__ init_frag_state(frag_init_state_s* const s, + uint32_t fragment_size, + int part_end_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, 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; + + s->frag.start_value_idx = row_to_value_idx(s->frag.start_row, s->col); + auto const end_value_idx = row_to_value_idx(s->frag.start_row + s->frag.num_rows, s->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 + // 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. + auto const first_level_val_idx = s->col.level_offsets[s->frag.start_row]; + auto const 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; + } +} + +template +void __device__ calculate_frag_size(frag_init_state_s* const s, int t) +{ + using block_reduce = cub::BlockReduce; + __shared__ typename block_reduce::TempStorage reduce_storage; + + auto const physical_type = s->col.physical_type; + auto const leaf_type = s->col.leaf_column->type().id(); + auto const dtype_len = physical_type_len(physical_type, leaf_type); + auto const nvals = s->frag.num_leaf_values; + auto const start_value_idx = s->frag.start_value_idx; + + for (uint32_t i = 0; i < nvals; i += block_size) { + auto const val_idx = start_value_idx + i + t; + auto const is_valid = i + t < nvals && val_idx < s->col.leaf_column->size() && + s->col.leaf_column->is_valid(val_idx); + uint32_t len; + if (is_valid) { + len = dtype_len; + if (physical_type == BYTE_ARRAY) { + switch (leaf_type) { + case type_id::STRING: { + auto str = s->col.leaf_column->element(val_idx); + len += str.size_bytes(); + } break; + case type_id::LIST: { + auto list_element = + get_element(*s->col.leaf_column, val_idx); + len += list_element.size_bytes(); + } break; + default: CUDF_UNREACHABLE("Unsupported data type for leaf column"); + } + } + } else { + len = 0; + } + + len = block_reduce(reduce_storage).Sum(len); + if (t == 0) { s->frag.fragment_data_size += len; } + __syncthreads(); + // page fragment size must fit in a 32-bit signed integer + if (s->frag.fragment_data_size > std::numeric_limits::max()) { + CUDF_UNREACHABLE("page fragment size exceeds maximum for i32"); + } + } +} + } // anonymous namespace // blockDim {512,1,1} 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) + gpuInitRowGroupFragments(device_2dspan frag, + device_span col_desc, + device_span partitions, + device_span part_frag_offset, + uint32_t fragment_size) { __shared__ __align__(16) frag_init_state_s state_g; - using block_reduce = cub::BlockReduce; - __shared__ typename block_reduce::TempStorage reduce_storage; - frag_init_state_s* const s = &state_g; uint32_t const t = threadIdx.x; - auto const physical_type = col_desc[blockIdx.x].physical_type; uint32_t const num_fragments_per_column = frag.size().second; if (t == 0) { s->col = col_desc[blockIdx.x]; } __syncthreads(); - auto const leaf_type = s->col.leaf_column->type().id(); - auto const dtype_len = physical_type_len(physical_type, leaf_type); - for (uint32_t frag_y = blockIdx.y; frag_y < num_fragments_per_column; frag_y += gridDim.y) { if (t == 0) { // 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; + int const p = it - part_frag_offset.begin() - 1; + int const 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, 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; - - s->frag.start_value_idx = row_to_value_idx(s->frag.start_row, s->col); - size_type end_value_idx = row_to_value_idx(s->frag.start_row + s->frag.num_rows, s->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 - // 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[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; - } + s->frag.chunk = frag[blockIdx.x][frag_y].chunk; + init_frag_state(s, fragment_size, part_end_row); } __syncthreads(); - size_type nvals = s->frag.num_leaf_values; - 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; - uint32_t is_valid = (i + t < nvals && val_idx < s->col.leaf_column->size()) - ? s->col.leaf_column->is_valid(val_idx) - : 0; - uint32_t len; - if (is_valid) { - len = dtype_len; - if (physical_type == BYTE_ARRAY) { - switch (leaf_type) { - case type_id::STRING: { - auto str = s->col.leaf_column->element(val_idx); - len += str.size_bytes(); - } break; - case type_id::LIST: { - auto list_element = - get_element(*s->col.leaf_column, val_idx); - len += list_element.size_bytes(); - } break; - default: CUDF_UNREACHABLE("Unsupported data type for leaf column"); - } - } - } else { - len = 0; - } - - len = block_reduce(reduce_storage).Sum(len); - if (t == 0) { s->frag.fragment_data_size += len; } - __syncthreads(); - // page fragment size must fit in a 32-bit signed integer - if (s->frag.fragment_data_size > std::numeric_limits::max()) { - CUDF_UNREACHABLE("page fragment size exceeds maximum for i32"); - } - } + calculate_frag_size(s, t); __syncthreads(); if (t == 0) { frag[blockIdx.x][frag_y] = s->frag; } } } +// blockDim {512,1,1} +template +__global__ void __launch_bounds__(block_size) + gpuCalculatePageFragments(device_span frag, + device_span column_frag_sizes) +{ + __shared__ __align__(16) frag_init_state_s state_g; + + EncColumnChunk* const ck_g = frag[blockIdx.x].chunk; + frag_init_state_s* const s = &state_g; + uint32_t const t = threadIdx.x; + auto const fragment_size = column_frag_sizes[ck_g->col_desc_id]; + + if (t == 0) { s->col = *ck_g->col_desc; } + __syncthreads(); + + if (t == 0) { + int const part_end_row = ck_g->start_row + ck_g->num_rows; + s->frag.start_row = ck_g->start_row + (blockIdx.x - ck_g->first_fragment) * fragment_size; + s->frag.chunk = ck_g; + init_frag_state(s, fragment_size, part_end_row); + } + __syncthreads(); + + calculate_frag_size(s, t); + if (t == 0) { frag[blockIdx.x] = s->frag; } +} + // blockDim {128,1,1} __global__ void __launch_bounds__(128) - gpuInitFragmentStats(device_2dspan groups, - device_2dspan fragments, - device_span col_desc) + gpuInitFragmentStats(device_span groups, + device_span fragments) { - uint32_t const lane_id = threadIdx.x & WARP_MASK; - uint32_t const column_id = blockIdx.x; - uint32_t const num_fragments_per_column = fragments.size().second; - - uint32_t frag_id = blockIdx.y * 4 + (threadIdx.x / cudf::detail::warp_size); - while (frag_id < num_fragments_per_column) { + uint32_t const lane_id = threadIdx.x & WARP_MASK; + uint32_t const frag_id = blockIdx.x * 4 + (threadIdx.x / cudf::detail::warp_size); + if (frag_id < fragments.size()) { if (lane_id == 0) { statistics_group g; - g.col = &col_desc[column_id]; - g.start_row = fragments[column_id][frag_id].start_value_idx; - g.num_rows = fragments[column_id][frag_id].num_leaf_values; - groups[column_id][frag_id] = g; + auto* const ck_g = fragments[frag_id].chunk; + g.col = ck_g->col_desc; + g.start_row = fragments[frag_id].start_value_idx; + g.num_rows = fragments[frag_id].num_leaf_values; + groups[frag_id] = g; } - frag_id += gridDim.y * 4; } } @@ -389,7 +424,7 @@ __global__ void __launch_bounds__(128) if (num_rows >= ck_g.num_rows || (values_in_page > 0 && (page_size + fragment_data_size > this_max_page_size)) || - rows_in_page >= max_page_size_rows) { + rows_in_page + frag_g.num_rows > max_page_size_rows) { if (ck_g.use_dictionary) { // Additional byte to store entry bit width page_size = 1 + max_RLE_page_size(ck_g.dict_rle_bits, values_in_page); @@ -2057,33 +2092,35 @@ __global__ void __launch_bounds__(1) ck_g->column_index_size = static_cast(col_idx_end - ck_g->column_index_blob); } -void InitPageFragments(device_2dspan frag, - device_span col_desc, - device_span partitions, - device_span part_frag_offset, - uint32_t fragment_size, - rmm::cuda_stream_view stream) +void InitRowGroupFragments(device_2dspan frag, + device_span col_desc, + device_span partitions, + device_span part_frag_offset, + uint32_t fragment_size, + rmm::cuda_stream_view stream) { auto const num_columns = frag.size().first; auto const num_fragments_per_column = frag.size().second; auto const grid_y = std::min(static_cast(num_fragments_per_column), MAX_GRID_Y_SIZE); dim3 const dim_grid(num_columns, grid_y); // 1 threadblock per fragment - gpuInitPageFragments<512><<>>( + gpuInitRowGroupFragments<512><<>>( frag, col_desc, partitions, part_frag_offset, fragment_size); } -void InitFragmentStatistics(device_2dspan groups, - device_2dspan fragments, - device_span col_desc, +void CalculatePageFragments(device_span frag, + device_span column_frag_sizes, + rmm::cuda_stream_view stream) +{ + gpuCalculatePageFragments<512><<>>(frag, column_frag_sizes); +} + +void InitFragmentStatistics(device_span groups, + device_span fragments, rmm::cuda_stream_view stream) { - int const num_columns = col_desc.size(); - int const num_fragments_per_column = fragments.size().second; - auto const y_dim = - util::div_rounding_up_safe(num_fragments_per_column, 128 / cudf::detail::warp_size); - auto const grid_y = std::min(static_cast(y_dim), MAX_GRID_Y_SIZE); - dim3 const dim_grid(num_columns, grid_y); // 1 warp per fragment - gpuInitFragmentStats<<>>(groups, fragments, col_desc); + int const num_fragments = fragments.size(); + int const dim = util::div_rounding_up_safe(num_fragments, 128 / cudf::detail::warp_size); + gpuInitFragmentStats<<>>(groups, fragments); } void InitEncoderPages(device_2dspan chunks, diff --git a/cpp/src/io/parquet/parquet_gpu.hpp b/cpp/src/io/parquet/parquet_gpu.hpp index 9b156745e41..c91f182c4f4 100644 --- a/cpp/src/io/parquet/parquet_gpu.hpp +++ b/cpp/src/io/parquet/parquet_gpu.hpp @@ -480,8 +480,9 @@ void DecodePageData(hostdevice_vector& pages, rmm::cuda_stream_view stream); /** - * @brief Launches kernel for initializing encoder page fragments + * @brief Launches kernel for initializing encoder row group fragments * + * These fragments are used to calculate row group boundaries. * 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. * @@ -492,24 +493,38 @@ void DecodePageData(hostdevice_vector& pages, * @param[in] fragment_size Number of rows per fragment * @param[in] stream CUDA stream to use */ -void InitPageFragments(cudf::detail::device_2dspan frag, - device_span col_desc, - device_span partitions, - device_span first_frag_in_part, - uint32_t fragment_size, - rmm::cuda_stream_view stream); +void InitRowGroupFragments(cudf::detail::device_2dspan frag, + device_span col_desc, + device_span partitions, + device_span first_frag_in_part, + uint32_t fragment_size, + rmm::cuda_stream_view stream); + +/** + * @brief Launches kernel for calculating encoder page fragments with variable fragment sizes + * + * 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. + * + * This assumes an initial call to InitRowGroupFragments has been made. + * + * @param[out] frag Fragment array [fragment_id] + * @param[in] column_frag_sizes Number of rows per fragment per column [column_id] + * @param[in] stream CUDA stream to use + */ +void CalculatePageFragments(device_span frag, + device_span column_frag_sizes, + rmm::cuda_stream_view stream); /** - * @brief Launches kernel for initializing fragment statistics groups + * @brief Launches kernel for initializing fragment statistics groups with variable fragment sizes * - * @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[out] groups Statistics groups [total_fragments] + * @param[in] fragments Page fragments [total_fragments] * @param[in] stream CUDA stream to use */ -void InitFragmentStatistics(cudf::detail::device_2dspan groups, - cudf::detail::device_2dspan fragments, - device_span col_desc, +void InitFragmentStatistics(device_span groups, + device_span fragments, rmm::cuda_stream_view stream); /** diff --git a/cpp/src/io/parquet/writer_impl.cu b/cpp/src/io/parquet/writer_impl.cu index 13ec2d652a6..88176ee1901 100644 --- a/cpp/src/io/parquet/writer_impl.cu +++ b/cpp/src/io/parquet/writer_impl.cu @@ -31,6 +31,7 @@ #include #include +#include #include #include #include @@ -38,6 +39,7 @@ #include #include #include +#include #include #include @@ -85,6 +87,44 @@ parquet::Compression to_parquet_compression(compression_type compression) } } +size_type column_size(column_view const& column, rmm::cuda_stream_view stream) +{ + if (column.size() == 0) { return 0; } + + if (is_fixed_width(column.type())) { + return size_of(column.type()) * column.size(); + } else if (column.type().id() == type_id::STRING) { + auto const scol = strings_column_view(column); + return cudf::detail::get_value(scol.offsets(), column.size(), stream) - + cudf::detail::get_value(scol.offsets(), 0, stream); + } else if (column.type().id() == type_id::STRUCT) { + auto const scol = structs_column_view(column); + size_type ret = 0; + for (int i = 0; i < scol.num_children(); i++) { + ret += column_size(scol.get_sliced_child(i), stream); + } + return ret; + } else if (column.type().id() == type_id::LIST) { + auto const lcol = lists_column_view(column); + return column_size(lcol.get_sliced_child(stream), stream); + } + + CUDF_FAIL("Unexpected compound type"); +} + +// checks to see if the given column has a fixed size. This doesn't +// check every row, so assumes string and list columns are not fixed, even +// if each row is the same width. +// TODO: update this if FIXED_LEN_BYTE_ARRAY is ever supported for writes. +bool is_col_fixed_width(column_view const& column) +{ + if (column.type().id() == type_id::STRUCT) { + return std::all_of(column.child_begin(), column.child_end(), is_col_fixed_width); + } + + return is_fixed_width(column.type()); +} + } // namespace struct aggregate_writer_metadata { @@ -886,34 +926,33 @@ gpu::parquet_column_device_view parquet_column_view::get_device_view( return desc; } -void writer::impl::init_page_fragments(cudf::detail::hostdevice_2dvector& frag, - device_span col_desc, - host_span partitions, - device_span part_frag_offset, - uint32_t fragment_size) +void writer::impl::init_row_group_fragments( + cudf::detail::hostdevice_2dvector& frag, + device_span col_desc, + host_span partitions, + device_span part_frag_offset, + uint32_t fragment_size) { auto d_partitions = cudf::detail::make_device_uvector_async(partitions, stream); - gpu::InitPageFragments(frag, col_desc, d_partitions, part_frag_offset, fragment_size, stream); + gpu::InitRowGroupFragments(frag, col_desc, d_partitions, part_frag_offset, fragment_size, stream); frag.device_to_host(stream, true); } -void writer::impl::gather_fragment_statistics( - device_2dspan frag_stats_chunk, - device_2dspan frag, - device_span col_desc, - uint32_t num_fragments) +void writer::impl::calculate_page_fragments(device_span frag, + host_span frag_sizes) { - auto num_columns = col_desc.size(); - rmm::device_uvector frag_stats_group(num_fragments * num_columns, stream); - auto frag_stats_group_2dview = - device_2dspan(frag_stats_group.data(), num_columns, num_fragments); - - gpu::InitFragmentStatistics(frag_stats_group_2dview, frag, col_desc, stream); - detail::calculate_group_statistics(frag_stats_chunk.data(), - frag_stats_group.data(), - num_fragments * num_columns, - stream, - int96_timestamps); + auto d_frag_sz = cudf::detail::make_device_uvector_async(frag_sizes, stream); + gpu::CalculatePageFragments(frag, d_frag_sz, stream); +} + +void writer::impl::gather_fragment_statistics(device_span frag_stats, + device_span frags) +{ + rmm::device_uvector frag_stats_group(frag_stats.size(), stream); + + gpu::InitFragmentStatistics(frag_stats_group, frags, stream); + detail::calculate_group_statistics( + frag_stats.data(), frag_stats_group.data(), frag_stats.size(), stream, int96_timestamps); stream.synchronize(); } @@ -1407,23 +1446,63 @@ void writer::impl::write(table_view const& table, std::vector co }); // Init page fragments - // 5000 is good enough for up to ~200-character strings. Longer strings will start producing - // fragments larger than the desired page size -> TODO: keep track of the max fragment size, and - // iteratively reduce this value if the largest fragment exceeds the max page size limit (we - // ideally want the page size to be below 1MB so as to have enough pages to get good - // compression/decompression performance). - // If using the default fragment size, scale it up or down depending on the requested page size. - if (max_page_fragment_size_ == cudf::io::default_max_page_fragment_size) { - max_page_fragment_size_ = (cudf::io::default_max_page_fragment_size * max_page_size_bytes) / - cudf::io::default_max_page_size_bytes; + // 5000 is good enough for up to ~200-character strings. Longer strings and deeply nested columns + // will start producing fragments larger than the desired page size, so calculate fragment sizes + // for each leaf column. Skip if the fragment size is not the default. + auto max_page_fragment_size = max_page_fragment_size_.value_or(default_max_page_fragment_size); + + std::vector column_frag_size(num_columns, max_page_fragment_size); + + if (table.num_rows() > 0 && not max_page_fragment_size_.has_value()) { + std::vector column_sizes; + std::transform(single_streams_table.begin(), + single_streams_table.end(), + std::back_inserter(column_sizes), + [this](auto const& column) { return column_size(column, stream); }); + + // adjust global fragment size if a single fragment will overrun a rowgroup + auto const table_size = std::reduce(column_sizes.begin(), column_sizes.end()); + auto const avg_row_len = util::div_rounding_up_safe(table_size, table.num_rows()); + if (avg_row_len > 0) { + auto const rg_frag_size = util::div_rounding_up_safe(max_row_group_size, avg_row_len); + max_page_fragment_size = std::min(rg_frag_size, max_page_fragment_size); + } + + // dividing page size by average row length will tend to overshoot the desired + // page size when there's high variability in the row lengths. instead, shoot + // for multiple fragments per page to smooth things out. using 2 was too + // unbalanced in final page sizes, so using 4 which seems to be a good + // compromise at smoothing things out without getting fragment sizes too small. + auto frag_size_fn = [&](auto const& col, size_type col_size) { + const int target_frags_per_page = is_col_fixed_width(col) ? 1 : 4; + auto const avg_len = + target_frags_per_page * util::div_rounding_up_safe(col_size, table.num_rows()); + if (avg_len > 0) { + auto const frag_size = util::div_rounding_up_safe(max_page_size_bytes, avg_len); + return std::min(max_page_fragment_size, frag_size); + } else { + return max_page_fragment_size; + } + }; + + std::transform(single_streams_table.begin(), + single_streams_table.end(), + column_sizes.begin(), + column_frag_size.begin(), + frag_size_fn); } + // Fragments are calculated in two passes. In the first pass, a uniform number of fragments + // per column is used. This is done to satisfy the requirement that each column chunk within + // a row group has the same number of rows. After the row group (and thus column chunk) + // boundaries are known, a second pass is done to calculate fragments to be used in determining + // page boundaries within each column chunk. std::vector num_frag_in_part; std::transform(partitions.begin(), partitions.end(), std::back_inserter(num_frag_in_part), - [this](auto const& part) { - return util::div_rounding_up_unsafe(part.num_rows, max_page_fragment_size_); + [this, max_page_fragment_size](auto const& part) { + 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()); @@ -1434,7 +1513,7 @@ void writer::impl::write(table_view const& table, std::vector co 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( + cudf::detail::hostdevice_2dvector row_group_fragments( num_columns, num_fragments, stream); if (num_fragments != 0) { @@ -1443,8 +1522,8 @@ void writer::impl::write(table_view const& table, std::vector co leaf_column_views = create_leaf_column_device_views( col_desc, *parent_column_table_device_view, stream); - init_page_fragments( - fragments, col_desc, partitions, d_part_frag_offset, max_page_fragment_size_); + init_row_group_fragments( + row_group_fragments, col_desc, partitions, d_part_frag_offset, max_page_fragment_size); } std::vector const global_rowgroup_base = md->num_row_groups_per_file(); @@ -1461,9 +1540,9 @@ void writer::impl::write(table_view const& table, std::vector co 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; + fragment_data_size += row_group_fragments[c][f].fragment_data_size; } - size_type fragment_num_rows = fragments[0][f].num_rows; + size_type fragment_num_rows = row_group_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 @@ -1490,17 +1569,6 @@ void writer::impl::write(table_view const& table, std::vector co } } - // Allocate column chunks and gather fragment statistics - rmm::device_uvector frag_stats(0, stream); - if (stats_granularity_ != statistics_freq::STATISTICS_NONE) { - frag_stats.resize(num_fragments * num_columns, stream); - 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); - } - } - 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); @@ -1509,6 +1577,9 @@ void writer::impl::write(table_view const& table, std::vector co auto const num_chunks = num_rowgroups * num_columns; hostdevice_2dvector chunks(num_rowgroups, num_columns, stream); + // total fragments per column (in case they are non-uniform) + std::vector frags_per_column(num_columns, 0); + for (size_t p = 0; p < partitions.size(); ++p) { int f = part_frag_offset[p]; size_type start_row = partitions[p].start_row; @@ -1516,22 +1587,21 @@ void writer::impl::write(table_view const& table, std::vector co size_t global_r = global_rowgroup_base[p] + r; // Number of rowgroups already in file/part 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_); + 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]; - 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 = {}; + ck.col_desc = col_desc.device_ptr() + c; + ck.col_desc_id = c; + ck.fragments = &row_group_fragments.device_view()[c][f]; + ck.stats = nullptr; 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); + auto chunk_fragments = row_group_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) { @@ -1551,15 +1621,23 @@ void writer::impl::write(table_view const& table, std::vector co 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; + + frags_per_column[c] += util::div_rounding_up_unsafe( + row_group.num_rows, std::min(column_frag_size[c], max_page_fragment_size)); } f += fragments_in_chunk; start_row += (uint32_t)row_group.num_rows; } } - fragments.host_to_device(stream); - auto dict_info_owner = build_chunk_dictionaries( - chunks, col_desc, fragments, compression_, dict_policy_, max_dictionary_size_, stream); + row_group_fragments.host_to_device(stream); + auto dict_info_owner = build_chunk_dictionaries(chunks, + col_desc, + row_group_fragments, + compression_, + dict_policy_, + max_dictionary_size_, + 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; @@ -1572,7 +1650,72 @@ void writer::impl::write(table_view const& table, std::vector co } } - // Build chunk dictionaries and count pages + // The code preceding this used a uniform fragment size for all columns. Now recompute + // fragments with a (potentially) varying number of fragments per column. + + // first figure out the total number of fragments and calculate the start offset for each column + std::vector frag_offsets; + size_type const total_frags = [&]() { + if (frags_per_column.size() > 0) { + std::exclusive_scan(frags_per_column.data(), + frags_per_column.data() + num_columns + 1, + std::back_inserter(frag_offsets), + 0); + return frag_offsets[num_columns]; + } else { + return 0; + } + }(); + + rmm::device_uvector frag_stats(0, stream); + hostdevice_vector page_fragments(total_frags, stream); + + // update fragments and/or prepare for fragment statistics calculation if necessary + if (total_frags != 0) { + if (stats_granularity_ != statistics_freq::STATISTICS_NONE) { + frag_stats.resize(total_frags, stream); + } + + for (int c = 0; c < num_columns; c++) { + auto frag_offset = frag_offsets[c]; + auto const frag_size = column_frag_size[c]; + + for (size_t p = 0; p < partitions.size(); ++p) { + for (int r = 0; r < num_rg_in_part[p]; r++) { + auto const global_r = global_rowgroup_base[p] + r; + auto const& row_group = md->file(p).row_groups[global_r]; + uint32_t const fragments_in_chunk = + util::div_rounding_up_unsafe(row_group.num_rows, frag_size); + gpu::EncColumnChunk& ck = chunks[r + first_rg_in_part[p]][c]; + ck.fragments = page_fragments.device_ptr(frag_offset); + ck.first_fragment = frag_offset; + + // update the chunk pointer here for each fragment in chunk.fragments + for (uint32_t i = 0; i < fragments_in_chunk; i++) { + page_fragments[frag_offset + i].chunk = + &chunks.device_view()[r + first_rg_in_part[p]][c]; + } + + if (not frag_stats.is_empty()) { ck.stats = frag_stats.data() + frag_offset; } + frag_offset += fragments_in_chunk; + } + } + } + + chunks.host_to_device(stream); + + // re-initialize page fragments + page_fragments.host_to_device(stream); + calculate_page_fragments(page_fragments, column_frag_size); + + // and gather fragment statistics + if (not frag_stats.is_empty()) { + gather_fragment_statistics(frag_stats, + {page_fragments.device_ptr(), static_cast(total_frags)}); + } + } + + // Build chunk dictionaries and count pages. Sends chunks to device. hostdevice_vector comp_page_sizes = init_page_sizes( chunks, col_desc, num_columns, max_page_size_bytes, max_page_size_rows, compression_, stream); diff --git a/cpp/src/io/parquet/writer_impl.hpp b/cpp/src/io/parquet/writer_impl.hpp index 3569281fb47..24c35455ff7 100644 --- a/cpp/src/io/parquet/writer_impl.hpp +++ b/cpp/src/io/parquet/writer_impl.hpp @@ -122,32 +122,42 @@ class writer::impl { private: /** - * @brief Gather page fragments + * @brief Gather row group fragments * - * @param frag Destination page fragments + * This calculates fragments to be used in determining row group boundariesa. + * + * @param frag Destination row group fragments * @param col_desc column description array * @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, - host_span partitions, - device_span part_frag_offset, - uint32_t fragment_size); + void init_row_group_fragments(hostdevice_2dvector& frag, + device_span col_desc, + host_span partitions, + device_span part_frag_offset, + uint32_t fragment_size); + + /** + * @brief Recalculate page fragments + * + * This calculates fragments to be used to determine page boundaries within + * column chunks. + * + * @param frag Destination page fragments + * @param frag_sizes Array of fragment sizes for each column + */ + void calculate_page_fragments(device_span frag, + host_span frag_sizes); /** * @brief Gather per-fragment statistics * - * @param dst_stats output statistics - * @param frag Input page fragments - * @param col_desc column description array - * @param num_fragments Total number of fragments per column + * @param frag_stats output statistics + * @param frags Input page fragments */ - void gather_fragment_statistics(device_2dspan dst_stats, - device_2dspan frag, - device_span col_desc, - uint32_t num_fragments); + void gather_fragment_statistics(device_span frag_stats, + device_span frags); /** * @brief Initialize encoder pages @@ -220,9 +230,9 @@ class writer::impl { statistics_freq stats_granularity_ = statistics_freq::STATISTICS_NONE; dictionary_policy dict_policy_ = dictionary_policy::ALWAYS; size_t max_dictionary_size_ = default_max_dictionary_size; - size_type max_page_fragment_size_ = default_max_page_fragment_size; bool int96_timestamps = false; int32_t column_index_truncate_length = default_column_index_truncate_length; + std::optional max_page_fragment_size_; // 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() diff --git a/cpp/tests/io/parquet_chunked_reader_test.cpp b/cpp/tests/io/parquet_chunked_reader_test.cpp index 0cecf62cc9d..ed95d8381d9 100644 --- a/cpp/tests/io/parquet_chunked_reader_test.cpp +++ b/cpp/tests/io/parquet_chunked_reader_test.cpp @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022, NVIDIA CORPORATION. + * Copyright (c) 2022-2023, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -93,6 +93,7 @@ auto write_file(std::vector>& input_columns, cudf::io::parquet_writer_options::builder(cudf::io::sink_info{filepath}, *input_table) .max_page_size_bytes(max_page_size_bytes) .max_page_size_rows(max_page_size_rows) + .max_page_fragment_size(cudf::io::default_max_page_fragment_size) .build(); cudf::io::write_parquet(write_opts); diff --git a/cpp/tests/io/parquet_test.cpp b/cpp/tests/io/parquet_test.cpp index 48f69e3ecd3..141c06733a6 100644 --- a/cpp/tests/io/parquet_test.cpp +++ b/cpp/tests/io/parquet_test.cpp @@ -3705,6 +3705,42 @@ TEST_F(ParquetWriterTest, CheckPageRows) EXPECT_EQ(ph.data_page_header.num_values, page_rows); } +TEST_F(ParquetWriterTest, CheckPageRowsAdjusted) +{ + // enough for a few pages with the default 20'000 rows/page + constexpr auto rows_per_page = 20'000; + constexpr auto num_rows = 3 * rows_per_page; + const std::string s1(32, 'a'); + auto col0_elements = + cudf::detail::make_counting_transform_iterator(0, [&](auto i) { return s1; }); + auto col0 = cudf::test::strings_column_wrapper(col0_elements, col0_elements + num_rows); + + auto const expected = table_view{{col0}}; + + auto const filepath = temp_env->get_temp_filepath("CheckPageRowsAdjusted.parquet"); + const cudf::io::parquet_writer_options out_opts = + cudf::io::parquet_writer_options::builder(cudf::io::sink_info{filepath}, expected) + .max_page_size_rows(rows_per_page); + cudf::io::write_parquet(out_opts); + + // check first page header and make sure it has only page_rows values + auto const source = cudf::io::datasource::create(filepath); + cudf::io::parquet::FileMetaData fmd; + + read_footer(source, &fmd); + CUDF_EXPECTS(fmd.row_groups.size() > 0, "No row groups found"); + CUDF_EXPECTS(fmd.row_groups[0].columns.size() == 1, "Invalid number of columns"); + auto const& first_chunk = fmd.row_groups[0].columns[0].meta_data; + CUDF_EXPECTS(first_chunk.data_page_offset > 0, "Invalid location for first data page"); + + // read first data page header. sizeof(PageHeader) is not exact, but the thrift encoded + // version should be smaller than size of the struct. + auto const ph = read_page_header( + source, {first_chunk.data_page_offset, sizeof(cudf::io::parquet::PageHeader), 0}); + + EXPECT_LE(ph.data_page_header.num_values, rows_per_page); +} + TEST_F(ParquetWriterTest, Decimal128Stats) { // check that decimal128 min and max statistics are written in network byte order @@ -4046,14 +4082,14 @@ int32_t compare_binary(const std::vector& v1, TEST_F(ParquetWriterTest, LargeColumnIndex) { // create a file large enough to be written in 2 batches (currently 1GB per batch) + // pick fragment size that num_rows is divisible by, so we'll get equal sized row groups const std::string s1(1000, 'a'); const std::string s2(1000, 'b'); - constexpr auto num_rows = 512 * 1024; + constexpr auto num_rows = 512 * 1024; + constexpr auto frag_size = num_rows / 128; - // TODO(ets) need dictionary_policy set to NEVER from #12211. Then - // we don't need to append a number to make the strings unique. auto col0_elements = cudf::detail::make_counting_transform_iterator( - 0, [&](auto i) { return ((i < num_rows) ? s1 : s2) + std::to_string(i); }); + 0, [&](auto i) { return (i < num_rows) ? s1 : s2; }); auto col0 = cudf::test::strings_column_wrapper(col0_elements, col0_elements + 2 * num_rows); auto const expected = table_view{{col0, col0}}; @@ -4063,6 +4099,8 @@ TEST_F(ParquetWriterTest, LargeColumnIndex) cudf::io::parquet_writer_options::builder(cudf::io::sink_info{filepath}, expected) .stats_level(cudf::io::statistics_freq::STATISTICS_COLUMN) .compression(cudf::io::compression_type::NONE) + .dictionary_policy(cudf::io::dictionary_policy::NEVER) + .max_page_fragment_size(frag_size) .row_group_size_bytes(1024 * 1024 * 1024) .row_group_size_rows(num_rows); cudf::io::write_parquet(out_opts); From a96b1508cf96207eef5e26b330d94d20f6bc5054 Mon Sep 17 00:00:00 2001 From: Vyas Ramasubramani Date: Wed, 22 Feb 2023 17:12:37 -0800 Subject: [PATCH 03/60] Stop using versioneer to manage versions (#12741) This PR replaces usage of versioneer with hard-coded version numbers in setup.py and __init__.py. Since cudf needs to manage versions across a wide range of file types (CMake, C++, Sphinx and doxygen docs, etc), versioneer cannot be relied on as a single source of truth and therefore does not allow us to single-source our versioning to the Git repo as is intended. Additionally, since the primary means of installing cudf is via conda packages (or now, pip packages), information from the package manager tends to be far more informative than the version strings for troubleshooting and debugging purposes. Conversely, the nonstandard version strings that it produces tend to be problematic for other tools, which at best will ignore such versions but at worst will simply fail. This PR also replaces usage of an environment variable to set the package name for wheels in setup.py, instead moving the renaming logic into the same sed script used to update package versions. This change makes setup.py essentially static, paving the way for migration to pyproject.toml. Authors: - Vyas Ramasubramani (https://github.com/vyasr) Approvers: - AJ Schmidt (https://github.com/ajschmidt8) - Lawrence Mitchell (https://github.com/wence-) URL: https://github.com/rapidsai/cudf/pull/12741 --- .gitattributes | 4 - .github/workflows/build.yaml | 2 + .github/workflows/pr.yaml | 2 + .pre-commit-config.yaml | 3 +- ci/checks/copyright.py | 3 +- ci/release/apply_wheel_modifications.sh | 33 + ci/release/update-version.sh | 27 +- python/cudf/_custom_build/backend.py | 37 - python/cudf/cudf/__init__.py | 4 +- python/cudf/cudf/_version.py | 566 ------- python/cudf/pyproject.toml | 6 +- python/cudf/setup.cfg | 12 - python/cudf/setup.py | 28 +- python/cudf/versioneer.py | 1904 ---------------------- python/cudf_kafka/cudf_kafka/_version.py | 566 ------- python/cudf_kafka/setup.cfg | 10 +- python/cudf_kafka/setup.py | 4 +- python/cudf_kafka/versioneer.py | 1904 ---------------------- python/custreamz/custreamz/_version.py | 566 ------- python/custreamz/setup.cfg | 10 +- python/custreamz/setup.py | 4 +- python/custreamz/versioneer.py | 1904 ---------------------- python/dask_cudf/dask_cudf/__init__.py | 6 +- python/dask_cudf/dask_cudf/_version.py | 566 ------- python/dask_cudf/setup.cfg | 10 +- python/dask_cudf/setup.py | 24 +- python/dask_cudf/versioneer.py | 1904 ---------------------- setup.cfg | 7 +- 28 files changed, 79 insertions(+), 10037 deletions(-) delete mode 100644 .gitattributes create mode 100755 ci/release/apply_wheel_modifications.sh delete mode 100644 python/cudf/_custom_build/backend.py delete mode 100644 python/cudf/cudf/_version.py delete mode 100644 python/cudf/versioneer.py delete mode 100644 python/cudf_kafka/cudf_kafka/_version.py delete mode 100644 python/cudf_kafka/versioneer.py delete mode 100644 python/custreamz/custreamz/_version.py delete mode 100644 python/custreamz/versioneer.py delete mode 100644 python/dask_cudf/dask_cudf/_version.py delete mode 100644 python/dask_cudf/versioneer.py diff --git a/.gitattributes b/.gitattributes deleted file mode 100644 index fbfe7434d50..00000000000 --- a/.gitattributes +++ /dev/null @@ -1,4 +0,0 @@ -python/cudf/cudf/_version.py export-subst -python/cudf_kafka/cudf_kafka/_version.py export-subst -python/custreamz/custreamz/_version.py export-subst -python/dask_cudf/dask_cudf/_version.py export-subst diff --git a/.github/workflows/build.yaml b/.github/workflows/build.yaml index 26d07515f70..fa6704ef04e 100644 --- a/.github/workflows/build.yaml +++ b/.github/workflows/build.yaml @@ -64,6 +64,7 @@ jobs: package-name: cudf package-dir: python/cudf skbuild-configure-options: "-DCUDF_BUILD_WHEELS=ON -DDETECT_CONDA_ENV=OFF" + uses-setup-env-vars: false wheel-publish-cudf: needs: wheel-build-cudf secrets: inherit @@ -85,6 +86,7 @@ jobs: date: ${{ inputs.date }} package-name: dask_cudf package-dir: python/dask_cudf + uses-setup-env-vars: false wheel-publish-dask-cudf: needs: wheel-build-dask-cudf secrets: inherit diff --git a/.github/workflows/pr.yaml b/.github/workflows/pr.yaml index f33fc15c52f..73df2de20c2 100644 --- a/.github/workflows/pr.yaml +++ b/.github/workflows/pr.yaml @@ -91,6 +91,7 @@ jobs: package-name: cudf package-dir: python/cudf skbuild-configure-options: "-DCUDF_BUILD_WHEELS=ON -DDETECT_CONDA_ENV=OFF" + uses-setup-env-vars: false wheel-tests-cudf: needs: wheel-build-cudf secrets: inherit @@ -112,6 +113,7 @@ jobs: package-name: dask_cudf package-dir: python/dask_cudf before-wheel: "RAPIDS_PY_WHEEL_NAME=cudf_cu11 rapids-download-wheels-from-s3 ./local-cudf && pip install --no-deps ./local-cudf/cudf*.whl" + uses-setup-env-vars: false wheel-tests-dask-cudf: needs: wheel-build-dask-cudf secrets: inherit diff --git a/.pre-commit-config.yaml b/.pre-commit-config.yaml index a13b4ca10f1..244fc0d3872 100644 --- a/.pre-commit-config.yaml +++ b/.pre-commit-config.yaml @@ -144,8 +144,7 @@ repos: exclude: | (?x)^( .*test.*| - ^CHANGELOG.md$| - ^.*versioneer.py$ + ^CHANGELOG.md$ ) default_language_version: diff --git a/ci/checks/copyright.py b/ci/checks/copyright.py index 0f2540c440c..e76d9524c76 100644 --- a/ci/checks/copyright.py +++ b/ci/checks/copyright.py @@ -1,4 +1,4 @@ -# Copyright (c) 2019-2022, NVIDIA CORPORATION. +# Copyright (c) 2019-2023, NVIDIA CORPORATION. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. @@ -31,7 +31,6 @@ ] ExemptFiles = [ re.compile(r"cpp/include/cudf_test/cxxopts.hpp"), - re.compile(r"versioneer[.]py"), ] # this will break starting at year 10000, which is probably OK :) diff --git a/ci/release/apply_wheel_modifications.sh b/ci/release/apply_wheel_modifications.sh new file mode 100755 index 00000000000..e017b24be6e --- /dev/null +++ b/ci/release/apply_wheel_modifications.sh @@ -0,0 +1,33 @@ +#!/bin/bash +# Copyright (c) 2023, NVIDIA CORPORATION. +# +# Usage: bash apply_wheel_modifications.sh + +VERSION=${1} +CUDA_SUFFIX=${2} + +# __init__.py versions +sed -i "s/__version__ = .*/__version__ = \"${VERSION}\"/g" python/cudf/cudf/__init__.py +sed -i "s/__version__ = .*/__version__ = \"${VERSION}\"/g" python/dask_cudf/dask_cudf/__init__.py +sed -i "s/__version__ = .*/__version__ = \"${VERSION}\"/g" python/cudf_kafka/cudf_kafka/__init__.py +sed -i "s/__version__ = .*/__version__ = \"${VERSION}\"/g" python/custreamz/custreamz/__init__.py + +# setup.py versions +sed -i "s/version=.*,/version=\"${VERSION}\",/g" python/cudf/setup.py +sed -i "s/version=.*,/version=\"${VERSION}\",/g" python/dask_cudf/setup.py +sed -i "s/version=.*,/version=\"${VERSION}\",/g" python/cudf_kafka/setup.py +sed -i "s/version=.*,/version=\"${VERSION}\",/g" python/custreamz/setup.py + +# cudf setup.py cuda suffixes +sed -i "s/name=\"cudf\"/name=\"cudf${CUDA_SUFFIX}\"/g" python/cudf/setup.py +sed -i "s/rmm/rmm${CUDA_SUFFIX}/g" python/cudf/setup.py +sed -i "s/ptxcompiler/ptxcompiler${CUDA_SUFFIX}/g" python/cudf/setup.py +sed -i "s/cubinlinker/cubinlinker${CUDA_SUFFIX}/g" python/cudf/setup.py + +# cudf pyproject.toml cuda suffixes +sed -i "s/rmm/rmm${CUDA_SUFFIX}/g" python/cudf/pyproject.toml + +# dask_cudf setup.py cuda suffixes +sed -i "s/name=\"dask-cudf\"/name=\"dask-cudf${CUDA_SUFFIX}\"/g" python/dask_cudf/setup.py +# Need to provide the == to avoid modifying the URL +sed -i "s/\"cudf==/\"cudf${CUDA_SUFFIX}==/g" python/dask_cudf/setup.py diff --git a/ci/release/update-version.sh b/ci/release/update-version.sh index 15d81127450..c8875fda641 100755 --- a/ci/release/update-version.sh +++ b/ci/release/update-version.sh @@ -34,19 +34,27 @@ function sed_runner() { # cpp update sed_runner 's/'"VERSION ${CURRENT_SHORT_TAG}.*"'/'"VERSION ${NEXT_FULL_TAG}"'/g' cpp/CMakeLists.txt -# cpp stream testing update -sed_runner 's/'"VERSION ${CURRENT_SHORT_TAG}.*"'/'"VERSION ${NEXT_FULL_TAG}"'/g' cpp/tests/utilities/identify_stream_usage/CMakeLists.txt - -# Python update +# Python CMakeLists updates sed_runner 's/'"cudf_version .*)"'/'"cudf_version ${NEXT_FULL_TAG})"'/g' python/cudf/CMakeLists.txt - # cpp libcudf_kafka update sed_runner 's/'"VERSION ${CURRENT_SHORT_TAG}.*"'/'"VERSION ${NEXT_FULL_TAG}"'/g' cpp/libcudf_kafka/CMakeLists.txt # cpp cudf_jni update sed_runner 's/'"VERSION ${CURRENT_SHORT_TAG}.*"'/'"VERSION ${NEXT_FULL_TAG}"'/g' java/src/main/native/CMakeLists.txt +# Python __init__.py updates +sed_runner "s/__version__ = .*/__version__ = \"${NEXT_FULL_TAG}\"/g" python/cudf/cudf/__init__.py +sed_runner "s/__version__ = .*/__version__ = \"${NEXT_FULL_TAG}\"/g" python/dask_cudf/dask_cudf/__init__.py +sed_runner "s/__version__ = .*/__version__ = \"${NEXT_FULL_TAG}\"/g" python/cudf_kafka/cudf_kafka/__init__.py +sed_runner "s/__version__ = .*/__version__ = \"${NEXT_FULL_TAG}\"/g" python/custreamz/custreamz/__init__.py + +# Python setup.py updates +sed_runner "s/version=.*,/version=\"${NEXT_FULL_TAG}\",/g" python/cudf/setup.py +sed_runner "s/version=.*,/version=\"${NEXT_FULL_TAG}\",/g" python/dask_cudf/setup.py +sed_runner "s/version=.*,/version=\"${NEXT_FULL_TAG}\",/g" python/cudf_kafka/setup.py +sed_runner "s/version=.*,/version=\"${NEXT_FULL_TAG}\",/g" python/custreamz/setup.py + # rapids-cmake version sed_runner 's/'"branch-.*\/RAPIDS.cmake"'/'"branch-${NEXT_SHORT_TAG}\/RAPIDS.cmake"'/g' fetch_rapids.cmake @@ -81,9 +89,12 @@ sed_runner "s/CUDF_TAG branch-${CURRENT_SHORT_TAG}/CUDF_TAG branch-${NEXT_SHORT_ # Need to distutils-normalize the original version NEXT_SHORT_TAG_PEP440=$(python -c "from setuptools.extern import packaging; print(packaging.version.Version('${NEXT_SHORT_TAG}'))") -# Wheel builds install intra-RAPIDS dependencies from same release -sed_runner "s/rmm{cuda_suffix}.*\",/rmm{cuda_suffix}==${NEXT_SHORT_TAG_PEP440}.*\",/g" python/cudf/setup.py -sed_runner "s/cudf{cuda_suffix}==.*\",/cudf{cuda_suffix}==${NEXT_SHORT_TAG_PEP440}.*\",/g" python/dask_cudf/setup.py +# Dependency versions in setup.py +sed_runner "s/rmm==.*\",/rmm==${NEXT_SHORT_TAG_PEP440}.*\",/g" python/cudf/setup.py +sed_runner "s/cudf==.*\",/cudf==${NEXT_SHORT_TAG_PEP440}.*\",/g" python/dask_cudf/setup.py + +# Dependency versions in pyproject.toml +sed_runner "s/rmm==.*\",/rmm==${NEXT_SHORT_TAG_PEP440}.*\",/g" python/pyproject.toml for FILE in .github/workflows/*.yaml; do sed_runner "/shared-action-workflows/ s/@.*/@branch-${NEXT_SHORT_TAG}/g" "${FILE}" diff --git a/python/cudf/_custom_build/backend.py b/python/cudf/_custom_build/backend.py deleted file mode 100644 index 37b7edf2432..00000000000 --- a/python/cudf/_custom_build/backend.py +++ /dev/null @@ -1,37 +0,0 @@ -# Copyright (c) 2022, NVIDIA CORPORATION. - -"""Custom build backend for cudf to get versioned requirements. - -Based on https://setuptools.pypa.io/en/latest/build_meta.html -""" -import os -from functools import wraps - -from setuptools import build_meta as _orig - -# Alias the required bits -build_wheel = _orig.build_wheel -build_sdist = _orig.build_sdist - - -def replace_requirements(func): - @wraps(func) - def wrapper(config_settings=None): - orig_list = getattr(_orig, func.__name__)(config_settings) - append_list = [ - f"rmm{os.getenv('RAPIDS_PY_WHEEL_CUDA_SUFFIX', default='')}" - ] - return orig_list + append_list - - return wrapper - - -get_requires_for_build_wheel = replace_requirements( - _orig.get_requires_for_build_wheel -) -get_requires_for_build_sdist = replace_requirements( - _orig.get_requires_for_build_sdist -) -get_requires_for_build_editable = replace_requirements( - _orig.get_requires_for_build_editable -) diff --git a/python/cudf/cudf/__init__.py b/python/cudf/cudf/__init__.py index 05f61ee4f5a..04b64e18594 100644 --- a/python/cudf/cudf/__init__.py +++ b/python/cudf/cudf/__init__.py @@ -10,7 +10,6 @@ import rmm from cudf import api, core, datasets, testing -from cudf._version import get_versions from cudf.api.extensions import ( register_dataframe_accessor, register_index_accessor, @@ -112,8 +111,7 @@ rmm.register_reinitialize_hook(clear_cache) -__version__ = get_versions()["version"] -del get_versions +__version__ = "23.04.00" __all__ = [ "BaseIndex", diff --git a/python/cudf/cudf/_version.py b/python/cudf/cudf/_version.py deleted file mode 100644 index 60a2afed39b..00000000000 --- a/python/cudf/cudf/_version.py +++ /dev/null @@ -1,566 +0,0 @@ -# Copyright (c) 2020-2022, NVIDIA CORPORATION. -# This file helps to compute a version number in source trees obtained from -# git-archive tarball (such as those provided by githubs download-from-tag -# feature). Distribution tarballs (built by setup.py sdist) and build -# directories (produced by setup.py build) will contain a much shorter file -# that just contains the computed version number. - -# This file is released into the public domain. Generated by -# versioneer-0.18 (https://github.com/warner/python-versioneer) - -"""Git implementation of _version.py.""" - -import errno -import os -import re -import subprocess -import sys - - -def get_keywords(): - """Get the keywords needed to look up the version information.""" - # these strings will be replaced by git during git-archive. - # setup.py/versioneer.py will grep for the variable names, so they must - # each be defined on a line of their own. _version.py will just call - # get_keywords(). - git_refnames = "$Format:%d$" - git_full = "$Format:%H$" - git_date = "$Format:%ci$" - keywords = {"refnames": git_refnames, "full": git_full, "date": git_date} - return keywords - - -class VersioneerConfig: - """Container for Versioneer configuration parameters.""" - - -def get_config(): - """Create, populate and return the VersioneerConfig() object.""" - # these strings are filled in when 'setup.py versioneer' creates - # _version.py - cfg = VersioneerConfig() - cfg.VCS = "git" - cfg.style = "pep440" - cfg.tag_prefix = "v" - cfg.parentdir_prefix = "cudf-" - cfg.versionfile_source = "cudf/_version.py" - cfg.verbose = False - return cfg - - -class NotThisMethod(Exception): - """Exception raised if a method is not valid for the current scenario.""" - - -LONG_VERSION_PY = {} -HANDLERS = {} - - -def register_vcs_handler(vcs, method): # decorator - """Decorator to mark a method as the handler for a particular VCS.""" - - def decorate(f): - """Store f in HANDLERS[vcs][method].""" - if vcs not in HANDLERS: - HANDLERS[vcs] = {} - HANDLERS[vcs][method] = f - return f - - return decorate - - -def run_command( - commands, args, cwd=None, verbose=False, hide_stderr=False, env=None -): - """Call the given command(s).""" - assert isinstance(commands, list) - p = None - for c in commands: - try: - dispcmd = str([c] + args) - # remember shell=False, so use git.cmd on windows, not just git - p = subprocess.Popen( - [c] + args, - cwd=cwd, - env=env, - stdout=subprocess.PIPE, - stderr=(subprocess.PIPE if hide_stderr else None), - ) - break - except OSError: - e = sys.exc_info()[1] - if e.errno == errno.ENOENT: - continue - if verbose: - print("unable to run %s" % dispcmd) - print(e) - return None, None - else: - if verbose: - print(f"unable to find command, tried {commands}") - return None, None - stdout = p.communicate()[0].strip() - if sys.version_info[0] >= 3: - stdout = stdout.decode() - if p.returncode != 0: - if verbose: - print("unable to run %s (error)" % dispcmd) - print("stdout was %s" % stdout) - return None, p.returncode - return stdout, p.returncode - - -def versions_from_parentdir(parentdir_prefix, root, verbose): - """Try to determine the version from the parent directory name. - - Source tarballs conventionally unpack into a directory that includes both - the project name and a version string. We will also support searching up - two directory levels for an appropriately named parent directory - """ - rootdirs = [] - - for i in range(3): - dirname = os.path.basename(root) - if dirname.startswith(parentdir_prefix): - return { - "version": dirname[len(parentdir_prefix) :], - "full-revisionid": None, - "dirty": False, - "error": None, - "date": None, - } - else: - rootdirs.append(root) - root = os.path.dirname(root) # up a level - - if verbose: - print( - "Tried directories %s but none started with prefix %s" - % (str(rootdirs), parentdir_prefix) - ) - raise NotThisMethod("rootdir doesn't start with parentdir_prefix") - - -@register_vcs_handler("git", "get_keywords") -def git_get_keywords(versionfile_abs): - """Extract version information from the given file.""" - # the code embedded in _version.py can just fetch the value of these - # keywords. When used from setup.py, we don't want to import _version.py, - # so we do it with a regexp instead. This function is not used from - # _version.py. - keywords = {} - try: - f = open(versionfile_abs) - for line in f.readlines(): - if line.strip().startswith("git_refnames ="): - mo = re.search(r'=\s*"(.*)"', line) - if mo: - keywords["refnames"] = mo.group(1) - if line.strip().startswith("git_full ="): - mo = re.search(r'=\s*"(.*)"', line) - if mo: - keywords["full"] = mo.group(1) - if line.strip().startswith("git_date ="): - mo = re.search(r'=\s*"(.*)"', line) - if mo: - keywords["date"] = mo.group(1) - f.close() - except OSError: - pass - return keywords - - -@register_vcs_handler("git", "keywords") -def git_versions_from_keywords(keywords, tag_prefix, verbose): - """Get version information from git keywords.""" - if not keywords: - raise NotThisMethod("no keywords at all, weird") - date = keywords.get("date") - if date is not None: - # git-2.2.0 added "%cI", which expands to an ISO-8601 -compliant - # datestamp. However we prefer "%ci" (which expands to an "ISO-8601 - # -like" string, which we must then edit to make compliant), because - # it's been around since git-1.5.3, and it's too difficult to - # discover which version we're using, or to work around using an - # older one. - date = date.strip().replace(" ", "T", 1).replace(" ", "", 1) - refnames = keywords["refnames"].strip() - if refnames.startswith("$Format"): - if verbose: - print("keywords are unexpanded, not using") - raise NotThisMethod("unexpanded keywords, not a git-archive tarball") - refs = {r.strip() for r in refnames.strip("()").split(",")} - # starting in git-1.8.3, tags are listed as "tag: foo-1.0" instead of - # just "foo-1.0". If we see a "tag: " prefix, prefer those. - TAG = "tag: " - tags = {r[len(TAG) :] for r in refs if r.startswith(TAG)} - if not tags: - # Either we're using git < 1.8.3, or there really are no tags. We use - # a heuristic: assume all version tags have a digit. The old git %d - # expansion behaves like git log --decorate=short and strips out the - # refs/heads/ and refs/tags/ prefixes that would let us distinguish - # between branches and tags. By ignoring refnames without digits, we - # filter out many common branch names like "release" and - # "stabilization", as well as "HEAD" and "master". - tags = {r for r in refs if re.search(r"\d", r)} - if verbose: - print("discarding '%s', no digits" % ",".join(refs - tags)) - if verbose: - print("likely tags: %s" % ",".join(sorted(tags))) - for ref in sorted(tags): - # sorting will prefer e.g. "2.0" over "2.0rc1" - if ref.startswith(tag_prefix): - r = ref[len(tag_prefix) :] - if verbose: - print("picking %s" % r) - return { - "version": r, - "full-revisionid": keywords["full"].strip(), - "dirty": False, - "error": None, - "date": date, - } - # no suitable tags, so version is "0+unknown", but full hex is still there - if verbose: - print("no suitable tags, using unknown + full revision id") - return { - "version": "0+unknown", - "full-revisionid": keywords["full"].strip(), - "dirty": False, - "error": "no suitable tags", - "date": None, - } - - -@register_vcs_handler("git", "pieces_from_vcs") -def git_pieces_from_vcs(tag_prefix, root, verbose, run_command=run_command): - """Get version from 'git describe' in the root of the source tree. - - This only gets called if the git-archive 'subst' keywords were *not* - expanded, and _version.py hasn't already been rewritten with a short - version string, meaning we're inside a checked out source tree. - """ - GITS = ["git"] - if sys.platform == "win32": - GITS = ["git.cmd", "git.exe"] - - out, rc = run_command( - GITS, ["rev-parse", "--git-dir"], cwd=root, hide_stderr=True - ) - if rc != 0: - if verbose: - print("Directory %s not under git control" % root) - raise NotThisMethod("'git rev-parse --git-dir' returned error") - - # if there is a tag matching tag_prefix, this yields TAG-NUM-gHEX[-dirty] - # if there isn't one, this yields HEX[-dirty] (no NUM) - describe_out, rc = run_command( - GITS, - [ - "describe", - "--tags", - "--dirty", - "--always", - "--long", - "--match", - "%s*" % tag_prefix, - ], - cwd=root, - ) - # --long was added in git-1.5.5 - if describe_out is None: - raise NotThisMethod("'git describe' failed") - describe_out = describe_out.strip() - full_out, rc = run_command(GITS, ["rev-parse", "HEAD"], cwd=root) - if full_out is None: - raise NotThisMethod("'git rev-parse' failed") - full_out = full_out.strip() - - pieces = {} - pieces["long"] = full_out - pieces["short"] = full_out[:7] # maybe improved later - pieces["error"] = None - - # parse describe_out. It will be like TAG-NUM-gHEX[-dirty] or HEX[-dirty] - # TAG might have hyphens. - git_describe = describe_out - - # look for -dirty suffix - dirty = git_describe.endswith("-dirty") - pieces["dirty"] = dirty - if dirty: - git_describe = git_describe[: git_describe.rindex("-dirty")] - - # now we have TAG-NUM-gHEX or HEX - - if "-" in git_describe: - # TAG-NUM-gHEX - mo = re.search(r"^(.+)-(\d+)-g([0-9a-f]+)$", git_describe) - if not mo: - # unparseable. Maybe git-describe is misbehaving? - pieces["error"] = ( - "unable to parse git-describe output: '%s'" % describe_out - ) - return pieces - - # tag - full_tag = mo.group(1) - if not full_tag.startswith(tag_prefix): - if verbose: - fmt = "tag '%s' doesn't start with prefix '%s'" - print(fmt % (full_tag, tag_prefix)) - pieces[ - "error" - ] = f"tag '{full_tag}' doesn't start with prefix '{tag_prefix}'" - return pieces - pieces["closest-tag"] = full_tag[len(tag_prefix) :] - - # distance: number of commits since tag - pieces["distance"] = int(mo.group(2)) - - # commit: short hex revision ID - pieces["short"] = mo.group(3) - - else: - # HEX: no tags - pieces["closest-tag"] = None - count_out, rc = run_command( - GITS, ["rev-list", "HEAD", "--count"], cwd=root - ) - pieces["distance"] = int(count_out) # total number of commits - - # commit date: see ISO-8601 comment in git_versions_from_keywords() - date = run_command(GITS, ["show", "-s", "--format=%ci", "HEAD"], cwd=root)[ - 0 - ].strip() - pieces["date"] = date.strip().replace(" ", "T", 1).replace(" ", "", 1) - - return pieces - - -def plus_or_dot(pieces): - """Return a + if we don't already have one, else return a .""" - if "+" in pieces.get("closest-tag", ""): - return "." - return "+" - - -def render_pep440(pieces): - """Build up version string, with post-release "local version identifier". - - Our goal: TAG[+DISTANCE.gHEX[.dirty]] . Note that if you - get a tagged build and then dirty it, you'll get TAG+0.gHEX.dirty - - Exceptions: - 1: no tags. git_describe was just HEX. 0+untagged.DISTANCE.gHEX[.dirty] - """ - if pieces["closest-tag"]: - rendered = pieces["closest-tag"] - if pieces["distance"] or pieces["dirty"]: - rendered += plus_or_dot(pieces) - rendered += "%d.g%s" % (pieces["distance"], pieces["short"]) - if pieces["dirty"]: - rendered += ".dirty" - else: - # exception #1 - rendered = "0+untagged.%d.g%s" % (pieces["distance"], pieces["short"]) - if pieces["dirty"]: - rendered += ".dirty" - return rendered - - -def render_pep440_pre(pieces): - """TAG[.post.devDISTANCE] -- No -dirty. - - Exceptions: - 1: no tags. 0.post.devDISTANCE - """ - if pieces["closest-tag"]: - rendered = pieces["closest-tag"] - if pieces["distance"]: - rendered += ".post.dev%d" % pieces["distance"] - else: - # exception #1 - rendered = "0.post.dev%d" % pieces["distance"] - return rendered - - -def render_pep440_post(pieces): - """TAG[.postDISTANCE[.dev0]+gHEX] . - - The ".dev0" means dirty. Note that .dev0 sorts backwards - (a dirty tree will appear "older" than the corresponding clean one), - but you shouldn't be releasing software with -dirty anyways. - - Exceptions: - 1: no tags. 0.postDISTANCE[.dev0] - """ - if pieces["closest-tag"]: - rendered = pieces["closest-tag"] - if pieces["distance"] or pieces["dirty"]: - rendered += ".post%d" % pieces["distance"] - if pieces["dirty"]: - rendered += ".dev0" - rendered += plus_or_dot(pieces) - rendered += "g%s" % pieces["short"] - else: - # exception #1 - rendered = "0.post%d" % pieces["distance"] - if pieces["dirty"]: - rendered += ".dev0" - rendered += "+g%s" % pieces["short"] - return rendered - - -def render_pep440_old(pieces): - """TAG[.postDISTANCE[.dev0]] . - - The ".dev0" means dirty. - - Exceptions: - 1: no tags. 0.postDISTANCE[.dev0] - """ - if pieces["closest-tag"]: - rendered = pieces["closest-tag"] - if pieces["distance"] or pieces["dirty"]: - rendered += ".post%d" % pieces["distance"] - if pieces["dirty"]: - rendered += ".dev0" - else: - # exception #1 - rendered = "0.post%d" % pieces["distance"] - if pieces["dirty"]: - rendered += ".dev0" - return rendered - - -def render_git_describe(pieces): - """TAG[-DISTANCE-gHEX][-dirty]. - - Like 'git describe --tags --dirty --always'. - - Exceptions: - 1: no tags. HEX[-dirty] (note: no 'g' prefix) - """ - if pieces["closest-tag"]: - rendered = pieces["closest-tag"] - if pieces["distance"]: - rendered += "-%d-g%s" % (pieces["distance"], pieces["short"]) - else: - # exception #1 - rendered = pieces["short"] - if pieces["dirty"]: - rendered += "-dirty" - return rendered - - -def render_git_describe_long(pieces): - """TAG-DISTANCE-gHEX[-dirty]. - - Like 'git describe --tags --dirty --always -long'. - The distance/hash is unconditional. - - Exceptions: - 1: no tags. HEX[-dirty] (note: no 'g' prefix) - """ - if pieces["closest-tag"]: - rendered = pieces["closest-tag"] - rendered += "-%d-g%s" % (pieces["distance"], pieces["short"]) - else: - # exception #1 - rendered = pieces["short"] - if pieces["dirty"]: - rendered += "-dirty" - return rendered - - -def render(pieces, style): - """Render the given version pieces into the requested style.""" - if pieces["error"]: - return { - "version": "unknown", - "full-revisionid": pieces.get("long"), - "dirty": None, - "error": pieces["error"], - "date": None, - } - - if not style or style == "default": - style = "pep440" # the default - - if style == "pep440": - rendered = render_pep440(pieces) - elif style == "pep440-pre": - rendered = render_pep440_pre(pieces) - elif style == "pep440-post": - rendered = render_pep440_post(pieces) - elif style == "pep440-old": - rendered = render_pep440_old(pieces) - elif style == "git-describe": - rendered = render_git_describe(pieces) - elif style == "git-describe-long": - rendered = render_git_describe_long(pieces) - else: - raise ValueError("unknown style '%s'" % style) - - return { - "version": rendered, - "full-revisionid": pieces["long"], - "dirty": pieces["dirty"], - "error": None, - "date": pieces.get("date"), - } - - -def get_versions(): - """Get version information or return default if unable to do so.""" - # I am in _version.py, which lives at ROOT/VERSIONFILE_SOURCE. If we have - # __file__, we can work backwards from there to the root. Some - # py2exe/bbfreeze/non-CPython implementations don't do __file__, in which - # case we can only use expanded keywords. - - cfg = get_config() - verbose = cfg.verbose - - try: - return git_versions_from_keywords( - get_keywords(), cfg.tag_prefix, verbose - ) - except NotThisMethod: - pass - - try: - root = os.path.realpath(__file__) - # versionfile_source is the relative path from the top of the source - # tree (where the .git directory might live) to this file. Invert - # this to find the root from __file__. - for i in cfg.versionfile_source.split("/"): - root = os.path.dirname(root) - except NameError: - return { - "version": "0+unknown", - "full-revisionid": None, - "dirty": None, - "error": "unable to find root of source tree", - "date": None, - } - - try: - pieces = git_pieces_from_vcs(cfg.tag_prefix, root, verbose) - return render(pieces, cfg.style) - except NotThisMethod: - pass - - try: - if cfg.parentdir_prefix: - return versions_from_parentdir(cfg.parentdir_prefix, root, verbose) - except NotThisMethod: - pass - - return { - "version": "0+unknown", - "full-revisionid": None, - "dirty": None, - "error": "unable to compute version", - "date": None, - } diff --git a/python/cudf/pyproject.toml b/python/cudf/pyproject.toml index 57464e83282..49c4d83245f 100644 --- a/python/cudf/pyproject.toml +++ b/python/cudf/pyproject.toml @@ -1,7 +1,7 @@ # Copyright (c) 2021-2023, NVIDIA CORPORATION. [build-system] - +build-backend = "setuptools.build_meta" requires = [ "wheel", "setuptools", @@ -13,7 +13,5 @@ requires = [ # Hard pin the patch version used during the build. "pyarrow==10.0.1", "protoc-wheel", - "versioneer", + "rmm==23.4.*", ] -build-backend = "backend" -backend-path = ["_custom_build"] diff --git a/python/cudf/setup.cfg b/python/cudf/setup.cfg index 59dd5d0179e..8380da371f9 100644 --- a/python/cudf/setup.cfg +++ b/python/cudf/setup.cfg @@ -1,17 +1,5 @@ # Copyright (c) 2018-2023, NVIDIA CORPORATION. -# See the docstring in versioneer.py for instructions. Note that you must -# re-run 'versioneer.py setup' after changing this section, and commit the -# resulting files. - -[versioneer] -VCS = git -style = pep440 -versionfile_source = cudf/_version.py -versionfile_build = cudf/_version.py -tag_prefix = v -parentdir_prefix = cudf- - [isort] line_length=79 multi_line_output=3 diff --git a/python/cudf/setup.py b/python/cudf/setup.py index 88bc2cfae28..0150c4fe715 100644 --- a/python/cudf/setup.py +++ b/python/cudf/setup.py @@ -1,13 +1,8 @@ # Copyright (c) 2018-2023, NVIDIA CORPORATION. -import os - -import versioneer from setuptools import find_packages from skbuild import setup -cuda_suffix = os.getenv("RAPIDS_PY_WHEEL_CUDA_SUFFIX", default="") - install_requires = [ "cachetools", "cuda-python>=11.7.1,<12.0", @@ -21,9 +16,9 @@ "typing_extensions", # Allow floating minor versions for Arrow. "pyarrow==10", - f"rmm{cuda_suffix}==23.4.*", - f"ptxcompiler{cuda_suffix}", - f"cubinlinker{cuda_suffix}", + "rmm==23.4.*", + "ptxcompiler", + "cubinlinker", "cupy-cuda11x", ] @@ -43,21 +38,9 @@ ] } -if "RAPIDS_PY_WHEEL_VERSIONEER_OVERRIDE" in os.environ: - orig_get_versions = versioneer.get_versions - - version_override = os.environ["RAPIDS_PY_WHEEL_VERSIONEER_OVERRIDE"] - - def get_versions(): - data = orig_get_versions() - data["version"] = version_override - return data - - versioneer.get_versions = get_versions - setup( - name=f"cudf{cuda_suffix}", - version=versioneer.get_version(), + name="cudf", + version="23.04.00", description="cuDF - GPU Dataframe", url="https://github.com/rapidsai/cudf", author="NVIDIA Corporation", @@ -72,7 +55,6 @@ def get_versions(): "Programming Language :: Python :: 3.9", "Programming Language :: Python :: 3.10", ], - cmdclass=versioneer.get_cmdclass(), include_package_data=True, packages=find_packages(include=["cudf", "cudf.*"]), package_data={ diff --git a/python/cudf/versioneer.py b/python/cudf/versioneer.py deleted file mode 100644 index a6537a34ede..00000000000 --- a/python/cudf/versioneer.py +++ /dev/null @@ -1,1904 +0,0 @@ -# Version: 0.18 - -"""The Versioneer - like a rocketeer, but for versions. - -The Versioneer -============== - -* like a rocketeer, but for versions! -* https://github.com/warner/python-versioneer -* Brian Warner -* License: Public Domain -* Compatible With: python2.6, 2.7, 3.2, 3.3, 3.4, 3.5, 3.6, and pypy -* [![Latest Version] -(https://pypip.in/version/versioneer/badge.svg?style=flat) -](https://pypi.python.org/pypi/versioneer/) -* [![Build Status] -(https://travis-ci.org/warner/python-versioneer.png?branch=master) -](https://travis-ci.org/warner/python-versioneer) - -This is a tool for managing a recorded version number in distutils-based -python projects. The goal is to remove the tedious and error-prone "update -the embedded version string" step from your release process. Making a new -release should be as easy as recording a new tag in your version-control -system, and maybe making new tarballs. - - -## Quick Install - -* `pip install versioneer` to somewhere to your $PATH -* add a `[versioneer]` section to your setup.cfg (see below) -* run `versioneer install` in your source tree, commit the results - -## Version Identifiers - -Source trees come from a variety of places: - -* a version-control system checkout (mostly used by developers) -* a nightly tarball, produced by build automation -* a snapshot tarball, produced by a web-based VCS browser, like github's - "tarball from tag" feature -* a release tarball, produced by "setup.py sdist", distributed through PyPI - -Within each source tree, the version identifier (either a string or a number, -this tool is format-agnostic) can come from a variety of places: - -* ask the VCS tool itself, e.g. "git describe" (for checkouts), which knows - about recent "tags" and an absolute revision-id -* the name of the directory into which the tarball was unpacked -* an expanded VCS keyword ($Id$, etc) -* a `_version.py` created by some earlier build step - -For released software, the version identifier is closely related to a VCS -tag. Some projects use tag names that include more than just the version -string (e.g. "myproject-1.2" instead of just "1.2"), in which case the tool -needs to strip the tag prefix to extract the version identifier. For -unreleased software (between tags), the version identifier should provide -enough information to help developers recreate the same tree, while also -giving them an idea of roughly how old the tree is (after version 1.2, before -version 1.3). Many VCS systems can report a description that captures this, -for example `git describe --tags --dirty --always` reports things like -"0.7-1-g574ab98-dirty" to indicate that the checkout is one revision past the -0.7 tag, has a unique revision id of "574ab98", and is "dirty" (it has -uncommitted changes. - -The version identifier is used for multiple purposes: - -* to allow the module to self-identify its version: `myproject.__version__` -* to choose a name and prefix for a 'setup.py sdist' tarball - -## Theory of Operation - -Versioneer works by adding a special `_version.py` file into your source -tree, where your `__init__.py` can import it. This `_version.py` knows how to -dynamically ask the VCS tool for version information at import time. - -`_version.py` also contains `$Revision$` markers, and the installation -process marks `_version.py` to have this marker rewritten with a tag name -during the `git archive` command. As a result, generated tarballs will -contain enough information to get the proper version. - -To allow `setup.py` to compute a version too, a `versioneer.py` is added to -the top level of your source tree, next to `setup.py` and the `setup.cfg` -that configures it. This overrides several distutils/setuptools commands to -compute the version when invoked, and changes `setup.py build` and `setup.py -sdist` to replace `_version.py` with a small static file that contains just -the generated version data. - -## Installation - -See [INSTALL.md](./INSTALL.md) for detailed installation instructions. - -## Version-String Flavors - -Code which uses Versioneer can learn about its version string at runtime by -importing `_version` from your main `__init__.py` file and running the -`get_versions()` function. From the "outside" (e.g. in `setup.py`), you can -import the top-level `versioneer.py` and run `get_versions()`. - -Both functions return a dictionary with different flavors of version -information: - -* `['version']`: A condensed version string, rendered using the selected - style. This is the most commonly used value for the project's version - string. The default "pep440" style yields strings like `0.11`, - `0.11+2.g1076c97`, or `0.11+2.g1076c97.dirty`. See the "Styles" section - below for alternative styles. - -* `['full-revisionid']`: detailed revision identifier. For Git, this is the - full SHA1 commit id, e.g. "1076c978a8d3cfc70f408fe5974aa6c092c949ac". - -* `['date']`: Date and time of the latest `HEAD` commit. For Git, it is the - commit date in ISO 8601 format. This will be None if the date is not - available. - -* `['dirty']`: a boolean, True if the tree has uncommitted changes. Note that - this is only accurate if run in a VCS checkout, otherwise it is likely to - be False or None - -* `['error']`: if the version string could not be computed, this will be set - to a string describing the problem, otherwise it will be None. It may be - useful to throw an exception in setup.py if this is set, to avoid e.g. - creating tarballs with a version string of "unknown". - -Some variants are more useful than others. Including `full-revisionid` in a -bug report should allow developers to reconstruct the exact code being tested -(or indicate the presence of local changes that should be shared with the -developers). `version` is suitable for display in an "about" box or a CLI -`--version` output: it can be easily compared against release notes and lists -of bugs fixed in various releases. - -The installer adds the following text to your `__init__.py` to place a basic -version in `YOURPROJECT.__version__`: - - from cudf._version import get_versions - __version__ = get_versions()['version'] - del get_versions - -## Styles - -The setup.cfg `style=` configuration controls how the VCS information is -rendered into a version string. - -The default style, "pep440", produces a PEP440-compliant string, equal to the -un-prefixed tag name for actual releases, and containing an additional "local -version" section with more detail for in-between builds. For Git, this is -TAG[+DISTANCE.gHEX[.dirty]] , using information from `git describe --tags ---dirty --always`. For example "0.11+2.g1076c97.dirty" indicates that the -tree is like the "1076c97" commit but has uncommitted changes (".dirty"), and -that this commit is two revisions ("+2") beyond the "0.11" tag. For released -software (exactly equal to a known tag), the identifier will only contain the -stripped tag, e.g. "0.11". - -Other styles are available. See [details.md](details.md) in the Versioneer -source tree for descriptions. - -## Debugging - -Versioneer tries to avoid fatal errors: if something goes wrong, it will tend -to return a version of "0+unknown". To investigate the problem, run `setup.py -version`, which will run the version-lookup code in a verbose mode, and will -display the full contents of `get_versions()` (including the `error` string, -which may help identify what went wrong). - -## Known Limitations - -Some situations are known to cause problems for Versioneer. This details the -most significant ones. More can be found on Github -[issues page](https://github.com/warner/python-versioneer/issues). - -### Subprojects - -Versioneer has limited support for source trees in which `setup.py` is not in -the root directory (e.g. `setup.py` and `.git/` are *not* siblings). The are -two common reasons why `setup.py` might not be in the root: - -* Source trees which contain multiple subprojects, such as - [Buildbot](https://github.com/buildbot/buildbot), which contains both - "master" and "slave" subprojects, each with their own `setup.py`, - `setup.cfg`, and `tox.ini`. Projects like these produce multiple PyPI - distributions (and upload multiple independently-installable tarballs). -* Source trees whose main purpose is to contain a C library, but which also - provide bindings to Python (and perhaps other langauges) in subdirectories. - -Versioneer will look for `.git` in parent directories, and most operations -should get the right version string. However `pip` and `setuptools` have bugs -and implementation details which frequently cause `pip install .` from a -subproject directory to fail to find a correct version string (so it usually -defaults to `0+unknown`). - -`pip install --editable .` should work correctly. `setup.py install` might -work too. - -Pip-8.1.1 is known to have this problem, but hopefully it will get fixed in -some later version. - -[Bug #38](https://github.com/warner/python-versioneer/issues/38) is tracking -this issue. The discussion in -[PR #61](https://github.com/warner/python-versioneer/pull/61) describes the -issue from the Versioneer side in more detail. -[pip PR#3176](https://github.com/pypa/pip/pull/3176) and -[pip PR#3615](https://github.com/pypa/pip/pull/3615) contain work to improve -pip to let Versioneer work correctly. - -Versioneer-0.16 and earlier only looked for a `.git` directory next to the -`setup.cfg`, so subprojects were completely unsupported with those releases. - -### Editable installs with setuptools <= 18.5 - -`setup.py develop` and `pip install --editable .` allow you to install a -project into a virtualenv once, then continue editing the source code (and -test) without re-installing after every change. - -"Entry-point scripts" (`setup(entry_points={"console_scripts": ..})`) are a -convenient way to specify executable scripts that should be installed along -with the python package. - -These both work as expected when using modern setuptools. When using -setuptools-18.5 or earlier, however, certain operations will cause -`pkg_resources.DistributionNotFound` errors when running the entrypoint -script, which must be resolved by re-installing the package. This happens -when the install happens with one version, then the egg_info data is -regenerated while a different version is checked out. Many setup.py commands -cause egg_info to be rebuilt (including `sdist`, `wheel`, and installing into -a different virtualenv), so this can be surprising. - -[Bug #83](https://github.com/warner/python-versioneer/issues/83) describes -this one, but upgrading to a newer version of setuptools should probably -resolve it. - -### Unicode version strings - -While Versioneer works (and is continually tested) with both Python 2 and -Python 3, it is not entirely consistent with bytes-vs-unicode distinctions. -Newer releases probably generate unicode version strings on py2. It's not -clear that this is wrong, but it may be surprising for applications when then -write these strings to a network connection or include them in bytes-oriented -APIs like cryptographic checksums. - -[Bug #71](https://github.com/warner/python-versioneer/issues/71) investigates -this question. - - -## Updating Versioneer - -To upgrade your project to a new release of Versioneer, do the following: - -* install the new Versioneer (`pip install -U versioneer` or equivalent) -* edit `setup.cfg`, if necessary, to include any new configuration settings - indicated by the release notes. See [UPGRADING](./UPGRADING.md) for details. -* re-run `versioneer install` in your source tree, to replace - `SRC/_version.py` -* commit any changed files - -## Future Directions - -This tool is designed to make it easily extended to other version-control -systems: all VCS-specific components are in separate directories like -src/git/ . The top-level `versioneer.py` script is assembled from these -components by running make-versioneer.py . In the future, make-versioneer.py -will take a VCS name as an argument, and will construct a version of -`versioneer.py` that is specific to the given VCS. It might also take the -configuration arguments that are currently provided manually during -installation by editing setup.py . Alternatively, it might go the other -direction and include code from all supported VCS systems, reducing the -number of intermediate scripts. - - -## License - -To make Versioneer easier to embed, all its code is dedicated to the public -domain. The `_version.py` that it creates is also in the public domain. -Specifically, both are released under the Creative Commons "Public Domain -Dedication" license (CC0-1.0), as described in -https://creativecommons.org/publicdomain/zero/1.0/ . - -""" - -from __future__ import print_function - -import errno -import json -import os -import re -import subprocess -import sys - -try: - import configparser -except ImportError: - import ConfigParser as configparser - - -class VersioneerConfig: - """Container for Versioneer configuration parameters.""" - - -def get_root(): - """Get the project root directory. - - We require that all commands are run from the project root, i.e. the - directory that contains setup.py, setup.cfg, and versioneer.py . - """ - root = os.path.realpath(os.path.abspath(os.getcwd())) - setup_py = os.path.join(root, "setup.py") - versioneer_py = os.path.join(root, "versioneer.py") - if not (os.path.exists(setup_py) or os.path.exists(versioneer_py)): - # allow 'python path/to/setup.py COMMAND' - root = os.path.dirname(os.path.realpath(os.path.abspath(sys.argv[0]))) - setup_py = os.path.join(root, "setup.py") - versioneer_py = os.path.join(root, "versioneer.py") - if not (os.path.exists(setup_py) or os.path.exists(versioneer_py)): - err = ( - "Versioneer was unable to run the project root directory. " - "Versioneer requires setup.py to be executed from " - "its immediate directory (like 'python setup.py COMMAND'), " - "or in a way that lets it use sys.argv[0] to find the root " - "(like 'python path/to/setup.py COMMAND')." - ) - raise VersioneerBadRootError(err) - try: - # Certain runtime workflows (setup.py install/develop in a setuptools - # tree) execute all dependencies in a single python process, so - # "versioneer" may be imported multiple times, and python's shared - # module-import table will cache the first one. So we can't use - # os.path.dirname(__file__), as that will find whichever - # versioneer.py was first imported, even in later projects. - me = os.path.realpath(os.path.abspath(__file__)) - me_dir = os.path.normcase(os.path.splitext(me)[0]) - vsr_dir = os.path.normcase(os.path.splitext(versioneer_py)[0]) - if me_dir != vsr_dir: - print( - "Warning: build in %s is using versioneer.py from %s" - % (os.path.dirname(me), versioneer_py) - ) - except NameError: - pass - return root - - -def get_config_from_root(root): - """Read the project setup.cfg file to determine Versioneer config.""" - # This might raise EnvironmentError (if setup.cfg is missing), or - # configparser.NoSectionError (if it lacks a [versioneer] section), or - # configparser.NoOptionError (if it lacks "VCS="). See the docstring at - # the top of versioneer.py for instructions on writing your setup.cfg . - setup_cfg = os.path.join(root, "setup.cfg") - parser = configparser.SafeConfigParser() - with open(setup_cfg, "r") as f: - parser.readfp(f) - VCS = parser.get("versioneer", "VCS") # mandatory - - def get(parser, name): - if parser.has_option("versioneer", name): - return parser.get("versioneer", name) - return None - - cfg = VersioneerConfig() - cfg.VCS = VCS - cfg.style = get(parser, "style") or "" - cfg.versionfile_source = get(parser, "versionfile_source") - cfg.versionfile_build = get(parser, "versionfile_build") - cfg.tag_prefix = get(parser, "tag_prefix") - if cfg.tag_prefix in ("''", '""'): - cfg.tag_prefix = "" - cfg.parentdir_prefix = get(parser, "parentdir_prefix") - cfg.verbose = get(parser, "verbose") - return cfg - - -class NotThisMethod(Exception): - """Exception raised if a method is not valid for the current scenario.""" - - -# these dictionaries contain VCS-specific tools -LONG_VERSION_PY = {} -HANDLERS = {} - - -def register_vcs_handler(vcs, method): # decorator - """Decorator to mark a method as the handler for a particular VCS.""" - - def decorate(f): - """Store f in HANDLERS[vcs][method].""" - if vcs not in HANDLERS: - HANDLERS[vcs] = {} - HANDLERS[vcs][method] = f - return f - - return decorate - - -def run_command( - commands, args, cwd=None, verbose=False, hide_stderr=False, env=None -): - """Call the given command(s).""" - assert isinstance(commands, list) - p = None - for c in commands: - try: - dispcmd = str([c] + args) - # remember shell=False, so use git.cmd on windows, not just git - p = subprocess.Popen( - [c] + args, - cwd=cwd, - env=env, - stdout=subprocess.PIPE, - stderr=(subprocess.PIPE if hide_stderr else None), - ) - break - except EnvironmentError: - e = sys.exc_info()[1] - if e.errno == errno.ENOENT: - continue - if verbose: - print("unable to run %s" % dispcmd) - print(e) - return None, None - else: - if verbose: - print("unable to find command, tried %s" % (commands,)) - return None, None - stdout = p.communicate()[0].strip() - if sys.version_info[0] >= 3: - stdout = stdout.decode() - if p.returncode != 0: - if verbose: - print("unable to run %s (error)" % dispcmd) - print("stdout was %s" % stdout) - return None, p.returncode - return stdout, p.returncode - - -LONG_VERSION_PY[ - "git" -] = r''' -# This file helps to compute a version number in source trees obtained from -# git-archive tarball (such as those provided by githubs download-from-tag -# feature). Distribution tarballs (built by setup.py sdist) and build -# directories (produced by setup.py build) will contain a much shorter file -# that just contains the computed version number. - -# This file is released into the public domain. Generated by -# versioneer-0.18 (https://github.com/warner/python-versioneer) - -"""Git implementation of _version.py.""" - -import errno -import os -import re -import subprocess -import sys - - -def get_keywords(): - """Get the keywords needed to look up the version information.""" - # these strings will be replaced by git during git-archive. - # setup.py/versioneer.py will grep for the variable names, so they must - # each be defined on a line of their own. _version.py will just call - # get_keywords(). - git_refnames = "%(DOLLAR)sFormat:%%d%(DOLLAR)s" - git_full = "%(DOLLAR)sFormat:%%H%(DOLLAR)s" - git_date = "%(DOLLAR)sFormat:%%ci%(DOLLAR)s" - keywords = {"refnames": git_refnames, "full": git_full, "date": git_date} - return keywords - - -class VersioneerConfig: - """Container for Versioneer configuration parameters.""" - - -def get_config(): - """Create, populate and return the VersioneerConfig() object.""" - # these strings are filled in when 'setup.py versioneer' creates - # _version.py - cfg = VersioneerConfig() - cfg.VCS = "git" - cfg.style = "%(STYLE)s" - cfg.tag_prefix = "%(TAG_PREFIX)s" - cfg.parentdir_prefix = "%(PARENTDIR_PREFIX)s" - cfg.versionfile_source = "%(VERSIONFILE_SOURCE)s" - cfg.verbose = False - return cfg - - -class NotThisMethod(Exception): - """Exception raised if a method is not valid for the current scenario.""" - - -LONG_VERSION_PY = {} -HANDLERS = {} - - -def register_vcs_handler(vcs, method): # decorator - """Decorator to mark a method as the handler for a particular VCS.""" - def decorate(f): - """Store f in HANDLERS[vcs][method].""" - if vcs not in HANDLERS: - HANDLERS[vcs] = {} - HANDLERS[vcs][method] = f - return f - return decorate - - -def run_command(commands, args, cwd=None, verbose=False, hide_stderr=False, - env=None): - """Call the given command(s).""" - assert isinstance(commands, list) - p = None - for c in commands: - try: - dispcmd = str([c] + args) - # remember shell=False, so use git.cmd on windows, not just git - p = subprocess.Popen([c] + args, cwd=cwd, env=env, - stdout=subprocess.PIPE, - stderr=(subprocess.PIPE if hide_stderr - else None)) - break - except EnvironmentError: - e = sys.exc_info()[1] - if e.errno == errno.ENOENT: - continue - if verbose: - print("unable to run %%s" %% dispcmd) - print(e) - return None, None - else: - if verbose: - print("unable to find command, tried %%s" %% (commands,)) - return None, None - stdout = p.communicate()[0].strip() - if sys.version_info[0] >= 3: - stdout = stdout.decode() - if p.returncode != 0: - if verbose: - print("unable to run %%s (error)" %% dispcmd) - print("stdout was %%s" %% stdout) - return None, p.returncode - return stdout, p.returncode - - -def versions_from_parentdir(parentdir_prefix, root, verbose): - """Try to determine the version from the parent directory name. - - Source tarballs conventionally unpack into a directory that includes both - the project name and a version string. We will also support searching up - two directory levels for an appropriately named parent directory - """ - rootdirs = [] - - for i in range(3): - dirname = os.path.basename(root) - if dirname.startswith(parentdir_prefix): - return {"version": dirname[len(parentdir_prefix):], - "full-revisionid": None, - "dirty": False, "error": None, "date": None} - else: - rootdirs.append(root) - root = os.path.dirname(root) # up a level - - if verbose: - print("Tried directories %%s but none started with prefix %%s" %% - (str(rootdirs), parentdir_prefix)) - raise NotThisMethod("rootdir doesn't start with parentdir_prefix") - - -@register_vcs_handler("git", "get_keywords") -def git_get_keywords(versionfile_abs): - """Extract version information from the given file.""" - # the code embedded in _version.py can just fetch the value of these - # keywords. When used from setup.py, we don't want to import _version.py, - # so we do it with a regexp instead. This function is not used from - # _version.py. - keywords = {} - try: - f = open(versionfile_abs, "r") - for line in f.readlines(): - if line.strip().startswith("git_refnames ="): - mo = re.search(r'=\s*"(.*)"', line) - if mo: - keywords["refnames"] = mo.group(1) - if line.strip().startswith("git_full ="): - mo = re.search(r'=\s*"(.*)"', line) - if mo: - keywords["full"] = mo.group(1) - if line.strip().startswith("git_date ="): - mo = re.search(r'=\s*"(.*)"', line) - if mo: - keywords["date"] = mo.group(1) - f.close() - except EnvironmentError: - pass - return keywords - - -@register_vcs_handler("git", "keywords") -def git_versions_from_keywords(keywords, tag_prefix, verbose): - """Get version information from git keywords.""" - if not keywords: - raise NotThisMethod("no keywords at all, weird") - date = keywords.get("date") - if date is not None: - # git-2.2.0 added "%%cI", which expands to an ISO-8601 -compliant - # datestamp. However we prefer "%%ci" (which expands to an "ISO-8601 - # -like" string, which we must then edit to make compliant), because - # it's been around since git-1.5.3, and it's too difficult to - # discover which version we're using, or to work around using an - # older one. - date = date.strip().replace(" ", "T", 1).replace(" ", "", 1) - refnames = keywords["refnames"].strip() - if refnames.startswith("$Format"): - if verbose: - print("keywords are unexpanded, not using") - raise NotThisMethod("unexpanded keywords, not a git-archive tarball") - refs = set([r.strip() for r in refnames.strip("()").split(",")]) - # starting in git-1.8.3, tags are listed as "tag: foo-1.0" instead of - # just "foo-1.0". If we see a "tag: " prefix, prefer those. - TAG = "tag: " - tags = set([r[len(TAG):] for r in refs if r.startswith(TAG)]) - if not tags: - # Either we're using git < 1.8.3, or there really are no tags. We use - # a heuristic: assume all version tags have a digit. The old git %%d - # expansion behaves like git log --decorate=short and strips out the - # refs/heads/ and refs/tags/ prefixes that would let us distinguish - # between branches and tags. By ignoring refnames without digits, we - # filter out many common branch names like "release" and - # "stabilization", as well as "HEAD" and "master". - tags = set([r for r in refs if re.search(r'\d', r)]) - if verbose: - print("discarding '%%s', no digits" %% ",".join(refs - tags)) - if verbose: - print("likely tags: %%s" %% ",".join(sorted(tags))) - for ref in sorted(tags): - # sorting will prefer e.g. "2.0" over "2.0rc1" - if ref.startswith(tag_prefix): - r = ref[len(tag_prefix):] - if verbose: - print("picking %%s" %% r) - return {"version": r, - "full-revisionid": keywords["full"].strip(), - "dirty": False, "error": None, - "date": date} - # no suitable tags, so version is "0+unknown", but full hex is still there - if verbose: - print("no suitable tags, using unknown + full revision id") - return {"version": "0+unknown", - "full-revisionid": keywords["full"].strip(), - "dirty": False, "error": "no suitable tags", "date": None} - - -@register_vcs_handler("git", "pieces_from_vcs") -def git_pieces_from_vcs(tag_prefix, root, verbose, run_command=run_command): - """Get version from 'git describe' in the root of the source tree. - - This only gets called if the git-archive 'subst' keywords were *not* - expanded, and _version.py hasn't already been rewritten with a short - version string, meaning we're inside a checked out source tree. - """ - GITS = ["git"] - if sys.platform == "win32": - GITS = ["git.cmd", "git.exe"] - - out, rc = run_command(GITS, ["rev-parse", "--git-dir"], cwd=root, - hide_stderr=True) - if rc != 0: - if verbose: - print("Directory %%s not under git control" %% root) - raise NotThisMethod("'git rev-parse --git-dir' returned error") - - # if there is a tag matching tag_prefix, this yields TAG-NUM-gHEX[-dirty] - # if there isn't one, this yields HEX[-dirty] (no NUM) - describe_out, rc = run_command(GITS, ["describe", "--tags", "--dirty", - "--always", "--long", - "--match", "%%s*" %% tag_prefix], - cwd=root) - # --long was added in git-1.5.5 - if describe_out is None: - raise NotThisMethod("'git describe' failed") - describe_out = describe_out.strip() - full_out, rc = run_command(GITS, ["rev-parse", "HEAD"], cwd=root) - if full_out is None: - raise NotThisMethod("'git rev-parse' failed") - full_out = full_out.strip() - - pieces = {} - pieces["long"] = full_out - pieces["short"] = full_out[:7] # maybe improved later - pieces["error"] = None - - # parse describe_out. It will be like TAG-NUM-gHEX[-dirty] or HEX[-dirty] - # TAG might have hyphens. - git_describe = describe_out - - # look for -dirty suffix - dirty = git_describe.endswith("-dirty") - pieces["dirty"] = dirty - if dirty: - git_describe = git_describe[:git_describe.rindex("-dirty")] - - # now we have TAG-NUM-gHEX or HEX - - if "-" in git_describe: - # TAG-NUM-gHEX - mo = re.search(r'^(.+)-(\d+)-g([0-9a-f]+)$', git_describe) - if not mo: - # unparseable. Maybe git-describe is misbehaving? - pieces["error"] = ("unable to parse git-describe output: '%%s'" - %% describe_out) - return pieces - - # tag - full_tag = mo.group(1) - if not full_tag.startswith(tag_prefix): - if verbose: - fmt = "tag '%%s' doesn't start with prefix '%%s'" - print(fmt %% (full_tag, tag_prefix)) - pieces["error"] = ("tag '%%s' doesn't start with prefix '%%s'" - %% (full_tag, tag_prefix)) - return pieces - pieces["closest-tag"] = full_tag[len(tag_prefix):] - - # distance: number of commits since tag - pieces["distance"] = int(mo.group(2)) - - # commit: short hex revision ID - pieces["short"] = mo.group(3) - - else: - # HEX: no tags - pieces["closest-tag"] = None - count_out, rc = run_command(GITS, ["rev-list", "HEAD", "--count"], - cwd=root) - pieces["distance"] = int(count_out) # total number of commits - - # commit date: see ISO-8601 comment in git_versions_from_keywords() - date = run_command(GITS, ["show", "-s", "--format=%%ci", "HEAD"], - cwd=root)[0].strip() - pieces["date"] = date.strip().replace(" ", "T", 1).replace(" ", "", 1) - - return pieces - - -def plus_or_dot(pieces): - """Return a + if we don't already have one, else return a .""" - if "+" in pieces.get("closest-tag", ""): - return "." - return "+" - - -def render_pep440(pieces): - """Build up version string, with post-release "local version identifier". - - Our goal: TAG[+DISTANCE.gHEX[.dirty]] . Note that if you - get a tagged build and then dirty it, you'll get TAG+0.gHEX.dirty - - Exceptions: - 1: no tags. git_describe was just HEX. 0+untagged.DISTANCE.gHEX[.dirty] - """ - if pieces["closest-tag"]: - rendered = pieces["closest-tag"] - if pieces["distance"] or pieces["dirty"]: - rendered += plus_or_dot(pieces) - rendered += "%%d.g%%s" %% (pieces["distance"], pieces["short"]) - if pieces["dirty"]: - rendered += ".dirty" - else: - # exception #1 - rendered = "0+untagged.%%d.g%%s" %% (pieces["distance"], - pieces["short"]) - if pieces["dirty"]: - rendered += ".dirty" - return rendered - - -def render_pep440_pre(pieces): - """TAG[.post.devDISTANCE] -- No -dirty. - - Exceptions: - 1: no tags. 0.post.devDISTANCE - """ - if pieces["closest-tag"]: - rendered = pieces["closest-tag"] - if pieces["distance"]: - rendered += ".post.dev%%d" %% pieces["distance"] - else: - # exception #1 - rendered = "0.post.dev%%d" %% pieces["distance"] - return rendered - - -def render_pep440_post(pieces): - """TAG[.postDISTANCE[.dev0]+gHEX] . - - The ".dev0" means dirty. Note that .dev0 sorts backwards - (a dirty tree will appear "older" than the corresponding clean one), - but you shouldn't be releasing software with -dirty anyways. - - Exceptions: - 1: no tags. 0.postDISTANCE[.dev0] - """ - if pieces["closest-tag"]: - rendered = pieces["closest-tag"] - if pieces["distance"] or pieces["dirty"]: - rendered += ".post%%d" %% pieces["distance"] - if pieces["dirty"]: - rendered += ".dev0" - rendered += plus_or_dot(pieces) - rendered += "g%%s" %% pieces["short"] - else: - # exception #1 - rendered = "0.post%%d" %% pieces["distance"] - if pieces["dirty"]: - rendered += ".dev0" - rendered += "+g%%s" %% pieces["short"] - return rendered - - -def render_pep440_old(pieces): - """TAG[.postDISTANCE[.dev0]] . - - The ".dev0" means dirty. - - Exceptions: - 1: no tags. 0.postDISTANCE[.dev0] - """ - if pieces["closest-tag"]: - rendered = pieces["closest-tag"] - if pieces["distance"] or pieces["dirty"]: - rendered += ".post%%d" %% pieces["distance"] - if pieces["dirty"]: - rendered += ".dev0" - else: - # exception #1 - rendered = "0.post%%d" %% pieces["distance"] - if pieces["dirty"]: - rendered += ".dev0" - return rendered - - -def render_git_describe(pieces): - """TAG[-DISTANCE-gHEX][-dirty]. - - Like 'git describe --tags --dirty --always'. - - Exceptions: - 1: no tags. HEX[-dirty] (note: no 'g' prefix) - """ - if pieces["closest-tag"]: - rendered = pieces["closest-tag"] - if pieces["distance"]: - rendered += "-%%d-g%%s" %% (pieces["distance"], pieces["short"]) - else: - # exception #1 - rendered = pieces["short"] - if pieces["dirty"]: - rendered += "-dirty" - return rendered - - -def render_git_describe_long(pieces): - """TAG-DISTANCE-gHEX[-dirty]. - - Like 'git describe --tags --dirty --always -long'. - The distance/hash is unconditional. - - Exceptions: - 1: no tags. HEX[-dirty] (note: no 'g' prefix) - """ - if pieces["closest-tag"]: - rendered = pieces["closest-tag"] - rendered += "-%%d-g%%s" %% (pieces["distance"], pieces["short"]) - else: - # exception #1 - rendered = pieces["short"] - if pieces["dirty"]: - rendered += "-dirty" - return rendered - - -def render(pieces, style): - """Render the given version pieces into the requested style.""" - if pieces["error"]: - return {"version": "unknown", - "full-revisionid": pieces.get("long"), - "dirty": None, - "error": pieces["error"], - "date": None} - - if not style or style == "default": - style = "pep440" # the default - - if style == "pep440": - rendered = render_pep440(pieces) - elif style == "pep440-pre": - rendered = render_pep440_pre(pieces) - elif style == "pep440-post": - rendered = render_pep440_post(pieces) - elif style == "pep440-old": - rendered = render_pep440_old(pieces) - elif style == "git-describe": - rendered = render_git_describe(pieces) - elif style == "git-describe-long": - rendered = render_git_describe_long(pieces) - else: - raise ValueError("unknown style '%%s'" %% style) - - return {"version": rendered, "full-revisionid": pieces["long"], - "dirty": pieces["dirty"], "error": None, - "date": pieces.get("date")} - - -def get_versions(): - """Get version information or return default if unable to do so.""" - # I am in _version.py, which lives at ROOT/VERSIONFILE_SOURCE. If we have - # __file__, we can work backwards from there to the root. Some - # py2exe/bbfreeze/non-CPython implementations don't do __file__, in which - # case we can only use expanded keywords. - - cfg = get_config() - verbose = cfg.verbose - - try: - return git_versions_from_keywords(get_keywords(), cfg.tag_prefix, - verbose) - except NotThisMethod: - pass - - try: - root = os.path.realpath(__file__) - # versionfile_source is the relative path from the top of the source - # tree (where the .git directory might live) to this file. Invert - # this to find the root from __file__. - for i in cfg.versionfile_source.split('/'): - root = os.path.dirname(root) - except NameError: - return {"version": "0+unknown", "full-revisionid": None, - "dirty": None, - "error": "unable to find root of source tree", - "date": None} - - try: - pieces = git_pieces_from_vcs(cfg.tag_prefix, root, verbose) - return render(pieces, cfg.style) - except NotThisMethod: - pass - - try: - if cfg.parentdir_prefix: - return versions_from_parentdir(cfg.parentdir_prefix, root, verbose) - except NotThisMethod: - pass - - return {"version": "0+unknown", "full-revisionid": None, - "dirty": None, - "error": "unable to compute version", "date": None} -''' - - -@register_vcs_handler("git", "get_keywords") -def git_get_keywords(versionfile_abs): - """Extract version information from the given file.""" - # the code embedded in _version.py can just fetch the value of these - # keywords. When used from setup.py, we don't want to import _version.py, - # so we do it with a regexp instead. This function is not used from - # _version.py. - keywords = {} - try: - f = open(versionfile_abs, "r") - for line in f.readlines(): - if line.strip().startswith("git_refnames ="): - mo = re.search(r'=\s*"(.*)"', line) - if mo: - keywords["refnames"] = mo.group(1) - if line.strip().startswith("git_full ="): - mo = re.search(r'=\s*"(.*)"', line) - if mo: - keywords["full"] = mo.group(1) - if line.strip().startswith("git_date ="): - mo = re.search(r'=\s*"(.*)"', line) - if mo: - keywords["date"] = mo.group(1) - f.close() - except EnvironmentError: - pass - return keywords - - -@register_vcs_handler("git", "keywords") -def git_versions_from_keywords(keywords, tag_prefix, verbose): - """Get version information from git keywords.""" - if not keywords: - raise NotThisMethod("no keywords at all, weird") - date = keywords.get("date") - if date is not None: - # git-2.2.0 added "%cI", which expands to an ISO-8601 -compliant - # datestamp. However we prefer "%ci" (which expands to an "ISO-8601 - # -like" string, which we must then edit to make compliant), because - # it's been around since git-1.5.3, and it's too difficult to - # discover which version we're using, or to work around using an - # older one. - date = date.strip().replace(" ", "T", 1).replace(" ", "", 1) - refnames = keywords["refnames"].strip() - if refnames.startswith("$Format"): - if verbose: - print("keywords are unexpanded, not using") - raise NotThisMethod("unexpanded keywords, not a git-archive tarball") - refs = set([r.strip() for r in refnames.strip("()").split(",")]) - # starting in git-1.8.3, tags are listed as "tag: foo-1.0" instead of - # just "foo-1.0". If we see a "tag: " prefix, prefer those. - TAG = "tag: " - tags = set([r[len(TAG) :] for r in refs if r.startswith(TAG)]) - if not tags: - # Either we're using git < 1.8.3, or there really are no tags. We use - # a heuristic: assume all version tags have a digit. The old git %d - # expansion behaves like git log --decorate=short and strips out the - # refs/heads/ and refs/tags/ prefixes that would let us distinguish - # between branches and tags. By ignoring refnames without digits, we - # filter out many common branch names like "release" and - # "stabilization", as well as "HEAD" and "master". - tags = set([r for r in refs if re.search(r"\d", r)]) - if verbose: - print("discarding '%s', no digits" % ",".join(refs - tags)) - if verbose: - print("likely tags: %s" % ",".join(sorted(tags))) - for ref in sorted(tags): - # sorting will prefer e.g. "2.0" over "2.0rc1" - if ref.startswith(tag_prefix): - r = ref[len(tag_prefix) :] - if verbose: - print("picking %s" % r) - return { - "version": r, - "full-revisionid": keywords["full"].strip(), - "dirty": False, - "error": None, - "date": date, - } - # no suitable tags, so version is "0+unknown", but full hex is still there - if verbose: - print("no suitable tags, using unknown + full revision id") - return { - "version": "0+unknown", - "full-revisionid": keywords["full"].strip(), - "dirty": False, - "error": "no suitable tags", - "date": None, - } - - -@register_vcs_handler("git", "pieces_from_vcs") -def git_pieces_from_vcs(tag_prefix, root, verbose, run_command=run_command): - """Get version from 'git describe' in the root of the source tree. - - This only gets called if the git-archive 'subst' keywords were *not* - expanded, and _version.py hasn't already been rewritten with a short - version string, meaning we're inside a checked out source tree. - """ - GITS = ["git"] - if sys.platform == "win32": - GITS = ["git.cmd", "git.exe"] - - out, rc = run_command( - GITS, ["rev-parse", "--git-dir"], cwd=root, hide_stderr=True - ) - if rc != 0: - if verbose: - print("Directory %s not under git control" % root) - raise NotThisMethod("'git rev-parse --git-dir' returned error") - - # if there is a tag matching tag_prefix, this yields TAG-NUM-gHEX[-dirty] - # if there isn't one, this yields HEX[-dirty] (no NUM) - describe_out, rc = run_command( - GITS, - [ - "describe", - "--tags", - "--dirty", - "--always", - "--long", - "--match", - "%s*" % tag_prefix, - ], - cwd=root, - ) - # --long was added in git-1.5.5 - if describe_out is None: - raise NotThisMethod("'git describe' failed") - describe_out = describe_out.strip() - full_out, rc = run_command(GITS, ["rev-parse", "HEAD"], cwd=root) - if full_out is None: - raise NotThisMethod("'git rev-parse' failed") - full_out = full_out.strip() - - pieces = {} - pieces["long"] = full_out - pieces["short"] = full_out[:7] # maybe improved later - pieces["error"] = None - - # parse describe_out. It will be like TAG-NUM-gHEX[-dirty] or HEX[-dirty] - # TAG might have hyphens. - git_describe = describe_out - - # look for -dirty suffix - dirty = git_describe.endswith("-dirty") - pieces["dirty"] = dirty - if dirty: - git_describe = git_describe[: git_describe.rindex("-dirty")] - - # now we have TAG-NUM-gHEX or HEX - - if "-" in git_describe: - # TAG-NUM-gHEX - mo = re.search(r"^(.+)-(\d+)-g([0-9a-f]+)$", git_describe) - if not mo: - # unparseable. Maybe git-describe is misbehaving? - pieces["error"] = ( - "unable to parse git-describe output: '%s'" % describe_out - ) - return pieces - - # tag - full_tag = mo.group(1) - if not full_tag.startswith(tag_prefix): - if verbose: - fmt = "tag '%s' doesn't start with prefix '%s'" - print(fmt % (full_tag, tag_prefix)) - pieces["error"] = "tag '%s' doesn't start with prefix '%s'" % ( - full_tag, - tag_prefix, - ) - return pieces - pieces["closest-tag"] = full_tag[len(tag_prefix) :] - - # distance: number of commits since tag - pieces["distance"] = int(mo.group(2)) - - # commit: short hex revision ID - pieces["short"] = mo.group(3) - - else: - # HEX: no tags - pieces["closest-tag"] = None - count_out, rc = run_command( - GITS, ["rev-list", "HEAD", "--count"], cwd=root - ) - pieces["distance"] = int(count_out) # total number of commits - - # commit date: see ISO-8601 comment in git_versions_from_keywords() - date = run_command(GITS, ["show", "-s", "--format=%ci", "HEAD"], cwd=root)[ - 0 - ].strip() - pieces["date"] = date.strip().replace(" ", "T", 1).replace(" ", "", 1) - - return pieces - - -def do_vcs_install(manifest_in, versionfile_source, ipy): - """Git-specific installation logic for Versioneer. - - For Git, this means creating/changing .gitattributes to mark _version.py - for export-subst keyword substitution. - """ - GITS = ["git"] - if sys.platform == "win32": - GITS = ["git.cmd", "git.exe"] - files = [manifest_in, versionfile_source] - if ipy: - files.append(ipy) - try: - me = __file__ - if me.endswith(".pyc") or me.endswith(".pyo"): - me = os.path.splitext(me)[0] + ".py" - versioneer_file = os.path.relpath(me) - except NameError: - versioneer_file = "versioneer.py" - files.append(versioneer_file) - present = False - try: - f = open(".gitattributes", "r") - for line in f.readlines(): - if line.strip().startswith(versionfile_source): - if "export-subst" in line.strip().split()[1:]: - present = True - f.close() - except EnvironmentError: - pass - if not present: - f = open(".gitattributes", "a+") - f.write("%s export-subst\n" % versionfile_source) - f.close() - files.append(".gitattributes") - run_command(GITS, ["add", "--"] + files) - - -def versions_from_parentdir(parentdir_prefix, root, verbose): - """Try to determine the version from the parent directory name. - - Source tarballs conventionally unpack into a directory that includes both - the project name and a version string. We will also support searching up - two directory levels for an appropriately named parent directory - """ - rootdirs = [] - - for i in range(3): - dirname = os.path.basename(root) - if dirname.startswith(parentdir_prefix): - return { - "version": dirname[len(parentdir_prefix) :], - "full-revisionid": None, - "dirty": False, - "error": None, - "date": None, - } - else: - rootdirs.append(root) - root = os.path.dirname(root) # up a level - - if verbose: - print( - "Tried directories %s but none started with prefix %s" - % (str(rootdirs), parentdir_prefix) - ) - raise NotThisMethod("rootdir doesn't start with parentdir_prefix") - - -SHORT_VERSION_PY = """ -# This file was generated by 'versioneer.py' (0.18) from -# revision-control system data, or from the parent directory name of an -# unpacked source archive. Distribution tarballs contain a pre-generated copy -# of this file. - -import json - -version_json = ''' -%s -''' # END VERSION_JSON - - -def get_versions(): - return json.loads(version_json) -""" - - -def versions_from_file(filename): - """Try to determine the version from _version.py if present.""" - try: - with open(filename) as f: - contents = f.read() - except EnvironmentError: - raise NotThisMethod("unable to read _version.py") - mo = re.search( - r"version_json = '''\n(.*)''' # END VERSION_JSON", - contents, - re.M | re.S, - ) - if not mo: - mo = re.search( - r"version_json = '''\r\n(.*)''' # END VERSION_JSON", - contents, - re.M | re.S, - ) - if not mo: - raise NotThisMethod("no version_json in _version.py") - return json.loads(mo.group(1)) - - -def write_to_version_file(filename, versions): - """Write the given version number to the given _version.py file.""" - os.unlink(filename) - contents = json.dumps( - versions, sort_keys=True, indent=1, separators=(",", ": ") - ) - with open(filename, "w") as f: - f.write(SHORT_VERSION_PY % contents) - - print("set %s to '%s'" % (filename, versions["version"])) - - -def plus_or_dot(pieces): - """Return a + if we don't already have one, else return a .""" - if "+" in pieces.get("closest-tag", ""): - return "." - return "+" - - -def render_pep440(pieces): - """Build up version string, with post-release "local version identifier". - - Our goal: TAG[+DISTANCE.gHEX[.dirty]] . Note that if you - get a tagged build and then dirty it, you'll get TAG+0.gHEX.dirty - - Exceptions: - 1: no tags. git_describe was just HEX. 0+untagged.DISTANCE.gHEX[.dirty] - """ - if pieces["closest-tag"]: - rendered = pieces["closest-tag"] - if pieces["distance"] or pieces["dirty"]: - rendered += plus_or_dot(pieces) - rendered += "%d.g%s" % (pieces["distance"], pieces["short"]) - if pieces["dirty"]: - rendered += ".dirty" - else: - # exception #1 - rendered = "0+untagged.%d.g%s" % (pieces["distance"], pieces["short"]) - if pieces["dirty"]: - rendered += ".dirty" - return rendered - - -def render_pep440_pre(pieces): - """TAG[.post.devDISTANCE] -- No -dirty. - - Exceptions: - 1: no tags. 0.post.devDISTANCE - """ - if pieces["closest-tag"]: - rendered = pieces["closest-tag"] - if pieces["distance"]: - rendered += ".post.dev%d" % pieces["distance"] - else: - # exception #1 - rendered = "0.post.dev%d" % pieces["distance"] - return rendered - - -def render_pep440_post(pieces): - """TAG[.postDISTANCE[.dev0]+gHEX] . - - The ".dev0" means dirty. Note that .dev0 sorts backwards - (a dirty tree will appear "older" than the corresponding clean one), - but you shouldn't be releasing software with -dirty anyways. - - Exceptions: - 1: no tags. 0.postDISTANCE[.dev0] - """ - if pieces["closest-tag"]: - rendered = pieces["closest-tag"] - if pieces["distance"] or pieces["dirty"]: - rendered += ".post%d" % pieces["distance"] - if pieces["dirty"]: - rendered += ".dev0" - rendered += plus_or_dot(pieces) - rendered += "g%s" % pieces["short"] - else: - # exception #1 - rendered = "0.post%d" % pieces["distance"] - if pieces["dirty"]: - rendered += ".dev0" - rendered += "+g%s" % pieces["short"] - return rendered - - -def render_pep440_old(pieces): - """TAG[.postDISTANCE[.dev0]] . - - The ".dev0" means dirty. - - Exceptions: - 1: no tags. 0.postDISTANCE[.dev0] - """ - if pieces["closest-tag"]: - rendered = pieces["closest-tag"] - if pieces["distance"] or pieces["dirty"]: - rendered += ".post%d" % pieces["distance"] - if pieces["dirty"]: - rendered += ".dev0" - else: - # exception #1 - rendered = "0.post%d" % pieces["distance"] - if pieces["dirty"]: - rendered += ".dev0" - return rendered - - -def render_git_describe(pieces): - """TAG[-DISTANCE-gHEX][-dirty]. - - Like 'git describe --tags --dirty --always'. - - Exceptions: - 1: no tags. HEX[-dirty] (note: no 'g' prefix) - """ - if pieces["closest-tag"]: - rendered = pieces["closest-tag"] - if pieces["distance"]: - rendered += "-%d-g%s" % (pieces["distance"], pieces["short"]) - else: - # exception #1 - rendered = pieces["short"] - if pieces["dirty"]: - rendered += "-dirty" - return rendered - - -def render_git_describe_long(pieces): - """TAG-DISTANCE-gHEX[-dirty]. - - Like 'git describe --tags --dirty --always -long'. - The distance/hash is unconditional. - - Exceptions: - 1: no tags. HEX[-dirty] (note: no 'g' prefix) - """ - if pieces["closest-tag"]: - rendered = pieces["closest-tag"] - rendered += "-%d-g%s" % (pieces["distance"], pieces["short"]) - else: - # exception #1 - rendered = pieces["short"] - if pieces["dirty"]: - rendered += "-dirty" - return rendered - - -def render(pieces, style): - """Render the given version pieces into the requested style.""" - if pieces["error"]: - return { - "version": "unknown", - "full-revisionid": pieces.get("long"), - "dirty": None, - "error": pieces["error"], - "date": None, - } - - if not style or style == "default": - style = "pep440" # the default - - if style == "pep440": - rendered = render_pep440(pieces) - elif style == "pep440-pre": - rendered = render_pep440_pre(pieces) - elif style == "pep440-post": - rendered = render_pep440_post(pieces) - elif style == "pep440-old": - rendered = render_pep440_old(pieces) - elif style == "git-describe": - rendered = render_git_describe(pieces) - elif style == "git-describe-long": - rendered = render_git_describe_long(pieces) - else: - raise ValueError("unknown style '%s'" % style) - - return { - "version": rendered, - "full-revisionid": pieces["long"], - "dirty": pieces["dirty"], - "error": None, - "date": pieces.get("date"), - } - - -class VersioneerBadRootError(Exception): - """The project root directory is unknown or missing key files.""" - - -def get_versions(verbose=False): - """Get the project version from whatever source is available. - - Returns dict with two keys: 'version' and 'full'. - """ - if "versioneer" in sys.modules: - # see the discussion in cmdclass.py:get_cmdclass() - del sys.modules["versioneer"] - - root = get_root() - cfg = get_config_from_root(root) - - assert cfg.VCS is not None, "please set [versioneer]VCS= in setup.cfg" - handlers = HANDLERS.get(cfg.VCS) - assert handlers, "unrecognized VCS '%s'" % cfg.VCS - verbose = verbose or cfg.verbose - assert ( - cfg.versionfile_source is not None - ), "please set versioneer.versionfile_source" - assert cfg.tag_prefix is not None, "please set versioneer.tag_prefix" - - versionfile_abs = os.path.join(root, cfg.versionfile_source) - - # extract version from first of: _version.py, VCS command (e.g. 'git - # describe'), parentdir. This is meant to work for developers using a - # source checkout, for users of a tarball created by 'setup.py sdist', - # and for users of a tarball/zipball created by 'git archive' or github's - # download-from-tag feature or the equivalent in other VCSes. - - get_keywords_f = handlers.get("get_keywords") - from_keywords_f = handlers.get("keywords") - if get_keywords_f and from_keywords_f: - try: - keywords = get_keywords_f(versionfile_abs) - ver = from_keywords_f(keywords, cfg.tag_prefix, verbose) - if verbose: - print("got version from expanded keyword %s" % ver) - return ver - except NotThisMethod: - pass - - try: - ver = versions_from_file(versionfile_abs) - if verbose: - print("got version from file %s %s" % (versionfile_abs, ver)) - return ver - except NotThisMethod: - pass - - from_vcs_f = handlers.get("pieces_from_vcs") - if from_vcs_f: - try: - pieces = from_vcs_f(cfg.tag_prefix, root, verbose) - ver = render(pieces, cfg.style) - if verbose: - print("got version from VCS %s" % ver) - return ver - except NotThisMethod: - pass - - try: - if cfg.parentdir_prefix: - ver = versions_from_parentdir(cfg.parentdir_prefix, root, verbose) - if verbose: - print("got version from parentdir %s" % ver) - return ver - except NotThisMethod: - pass - - if verbose: - print("unable to compute version") - - return { - "version": "0+unknown", - "full-revisionid": None, - "dirty": None, - "error": "unable to compute version", - "date": None, - } - - -def get_version(): - """Get the short version string for this project.""" - return get_versions()["version"] - - -def get_cmdclass(): - """Get the custom setuptools/distutils subclasses used by Versioneer.""" - if "versioneer" in sys.modules: - del sys.modules["versioneer"] - # this fixes the "python setup.py develop" case (also 'install' and - # 'easy_install .'), in which subdependencies of the main project are - # built (using setup.py bdist_egg) in the same python process. Assume - # a main project A and a dependency B, which use different versions - # of Versioneer. A's setup.py imports A's Versioneer, leaving it in - # sys.modules by the time B's setup.py is executed, causing B to run - # with the wrong versioneer. Setuptools wraps the sub-dep builds in a - # sandbox that restores sys.modules to it's pre-build state, so the - # parent is protected against the child's "import versioneer". By - # removing ourselves from sys.modules here, before the child build - # happens, we protect the child from the parent's versioneer too. - # Also see https://github.com/warner/python-versioneer/issues/52 - - cmds = {} - - # we add "version" to both distutils and setuptools - from distutils.core import Command - - class cmd_version(Command): - description = "report generated version string" - user_options = [] - boolean_options = [] - - def initialize_options(self): - pass - - def finalize_options(self): - pass - - def run(self): - vers = get_versions(verbose=True) - print("Version: %s" % vers["version"]) - print(" full-revisionid: %s" % vers.get("full-revisionid")) - print(" dirty: %s" % vers.get("dirty")) - print(" date: %s" % vers.get("date")) - if vers["error"]: - print(" error: %s" % vers["error"]) - - cmds["version"] = cmd_version - - # we override "build_py" in both distutils and setuptools - # - # most invocation pathways end up running build_py: - # distutils/build -> build_py - # distutils/install -> distutils/build ->.. - # setuptools/bdist_wheel -> distutils/install ->.. - # setuptools/bdist_egg -> distutils/install_lib -> build_py - # setuptools/install -> bdist_egg ->.. - # setuptools/develop -> ? - # pip install: - # copies source tree to a tempdir before running egg_info/etc - # if .git isn't copied too, 'git describe' will fail - # then does setup.py bdist_wheel, or sometimes setup.py install - # setup.py egg_info -> ? - - # we override different "build_py" commands for both environments - if "setuptools" in sys.modules: - from setuptools.command.build_py import build_py as _build_py - else: - from distutils.command.build_py import build_py as _build_py - - class cmd_build_py(_build_py): - def run(self): - root = get_root() - cfg = get_config_from_root(root) - versions = get_versions() - _build_py.run(self) - # now locate _version.py in the new build/ directory and replace - # it with an updated value - if cfg.versionfile_build: - target_versionfile = os.path.join( - self.build_lib, cfg.versionfile_build - ) - print("UPDATING %s" % target_versionfile) - write_to_version_file(target_versionfile, versions) - - cmds["build_py"] = cmd_build_py - - if "cx_Freeze" in sys.modules: # cx_freeze enabled? - from cx_Freeze.dist import build_exe as _build_exe - - # nczeczulin reports that py2exe won't like the pep440-style string - # as FILEVERSION, but it can be used for PRODUCTVERSION, e.g. - # setup(console=[{ - # "version": versioneer.get_version().split("+", 1)[0], # FILEVERSION - # "product_version": versioneer.get_version(), - # ... - - class cmd_build_exe(_build_exe): - def run(self): - root = get_root() - cfg = get_config_from_root(root) - versions = get_versions() - target_versionfile = cfg.versionfile_source - print("UPDATING %s" % target_versionfile) - write_to_version_file(target_versionfile, versions) - - _build_exe.run(self) - os.unlink(target_versionfile) - with open(cfg.versionfile_source, "w") as f: - LONG = LONG_VERSION_PY[cfg.VCS] - f.write( - LONG - % { - "DOLLAR": "$", - "STYLE": cfg.style, - "TAG_PREFIX": cfg.tag_prefix, - "PARENTDIR_PREFIX": cfg.parentdir_prefix, - "VERSIONFILE_SOURCE": cfg.versionfile_source, - } - ) - - cmds["build_exe"] = cmd_build_exe - del cmds["build_py"] - - if "py2exe" in sys.modules: # py2exe enabled? - try: - from py2exe.distutils_buildexe import py2exe as _py2exe # py3 - except ImportError: - from py2exe.build_exe import py2exe as _py2exe # py2 - - class cmd_py2exe(_py2exe): - def run(self): - root = get_root() - cfg = get_config_from_root(root) - versions = get_versions() - target_versionfile = cfg.versionfile_source - print("UPDATING %s" % target_versionfile) - write_to_version_file(target_versionfile, versions) - - _py2exe.run(self) - os.unlink(target_versionfile) - with open(cfg.versionfile_source, "w") as f: - LONG = LONG_VERSION_PY[cfg.VCS] - f.write( - LONG - % { - "DOLLAR": "$", - "STYLE": cfg.style, - "TAG_PREFIX": cfg.tag_prefix, - "PARENTDIR_PREFIX": cfg.parentdir_prefix, - "VERSIONFILE_SOURCE": cfg.versionfile_source, - } - ) - - cmds["py2exe"] = cmd_py2exe - - # we override different "sdist" commands for both environments - if "setuptools" in sys.modules: - from setuptools.command.sdist import sdist as _sdist - else: - from distutils.command.sdist import sdist as _sdist - - class cmd_sdist(_sdist): - def run(self): - versions = get_versions() - self._versioneer_generated_versions = versions - # unless we update this, the command will keep using the old - # version - self.distribution.metadata.version = versions["version"] - return _sdist.run(self) - - def make_release_tree(self, base_dir, files): - root = get_root() - cfg = get_config_from_root(root) - _sdist.make_release_tree(self, base_dir, files) - # now locate _version.py in the new base_dir directory - # (remembering that it may be a hardlink) and replace it with an - # updated value - target_versionfile = os.path.join(base_dir, cfg.versionfile_source) - print("UPDATING %s" % target_versionfile) - write_to_version_file( - target_versionfile, self._versioneer_generated_versions - ) - - cmds["sdist"] = cmd_sdist - - return cmds - - -CONFIG_ERROR = """ -setup.cfg is missing the necessary Versioneer configuration. You need -a section like: - - [versioneer] - VCS = git - style = pep440 - versionfile_source = src/myproject/_version.py - versionfile_build = myproject/_version.py - tag_prefix = - parentdir_prefix = myproject- - -You will also need to edit your setup.py to use the results: - - import versioneer - setup(version=versioneer.get_version(), - cmdclass=versioneer.get_cmdclass(), ...) - -Please read the docstring in ./versioneer.py for configuration instructions, -edit setup.cfg, and re-run the installer or 'python versioneer.py setup'. -""" - -SAMPLE_CONFIG = """ -# See the docstring in versioneer.py for instructions. Note that you must -# re-run 'versioneer.py setup' after changing this section, and commit the -# resulting files. - -[versioneer] -#VCS = git -#style = pep440 -#versionfile_source = -#versionfile_build = -#tag_prefix = -#parentdir_prefix = - -""" - -INIT_PY_SNIPPET = """ -from cudf._version import get_versions -__version__ = get_versions()['version'] -del get_versions -""" - - -def do_setup(): - """Main VCS-independent setup function for installing Versioneer.""" - root = get_root() - try: - cfg = get_config_from_root(root) - except ( - EnvironmentError, - configparser.NoSectionError, - configparser.NoOptionError, - ) as e: - if isinstance(e, (EnvironmentError, configparser.NoSectionError)): - print( - "Adding sample versioneer config to setup.cfg", file=sys.stderr - ) - with open(os.path.join(root, "setup.cfg"), "a") as f: - f.write(SAMPLE_CONFIG) - print(CONFIG_ERROR, file=sys.stderr) - return 1 - - print(" creating %s" % cfg.versionfile_source) - with open(cfg.versionfile_source, "w") as f: - LONG = LONG_VERSION_PY[cfg.VCS] - f.write( - LONG - % { - "DOLLAR": "$", - "STYLE": cfg.style, - "TAG_PREFIX": cfg.tag_prefix, - "PARENTDIR_PREFIX": cfg.parentdir_prefix, - "VERSIONFILE_SOURCE": cfg.versionfile_source, - } - ) - - ipy = os.path.join(os.path.dirname(cfg.versionfile_source), "__init__.py") - if os.path.exists(ipy): - try: - with open(ipy, "r") as f: - old = f.read() - except EnvironmentError: - old = "" - if INIT_PY_SNIPPET not in old: - print(" appending to %s" % ipy) - with open(ipy, "a") as f: - f.write(INIT_PY_SNIPPET) - else: - print(" %s unmodified" % ipy) - else: - print(" %s doesn't exist, ok" % ipy) - ipy = None - - # Make sure both the top-level "versioneer.py" and versionfile_source - # (PKG/_version.py, used by runtime code) are in MANIFEST.in, so - # they'll be copied into source distributions. Pip won't be able to - # install the package without this. - manifest_in = os.path.join(root, "MANIFEST.in") - simple_includes = set() - try: - with open(manifest_in, "r") as f: - for line in f: - if line.startswith("include "): - for include in line.split()[1:]: - simple_includes.add(include) - except EnvironmentError: - pass - # That doesn't cover everything MANIFEST.in can do - # (http://docs.python.org/2/distutils/sourcedist.html#commands), so - # it might give some false negatives. Appending redundant 'include' - # lines is safe, though. - if "versioneer.py" not in simple_includes: - print(" appending 'versioneer.py' to MANIFEST.in") - with open(manifest_in, "a") as f: - f.write("include versioneer.py\n") - else: - print(" 'versioneer.py' already in MANIFEST.in") - if cfg.versionfile_source not in simple_includes: - print( - " appending versionfile_source ('%s') to MANIFEST.in" - % cfg.versionfile_source - ) - with open(manifest_in, "a") as f: - f.write("include %s\n" % cfg.versionfile_source) - else: - print(" versionfile_source already in MANIFEST.in") - - # Make VCS-specific changes. For git, this means creating/changing - # .gitattributes to mark _version.py for export-subst keyword - # substitution. - do_vcs_install(manifest_in, cfg.versionfile_source, ipy) - return 0 - - -def scan_setup_py(): - """Validate the contents of setup.py against Versioneer's expectations.""" - found = set() - setters = False - errors = 0 - with open("setup.py", "r") as f: - for line in f.readlines(): - if "import versioneer" in line: - found.add("import") - if "versioneer.get_cmdclass()" in line: - found.add("cmdclass") - if "versioneer.get_version()" in line: - found.add("get_version") - if "versioneer.VCS" in line: - setters = True - if "versioneer.versionfile_source" in line: - setters = True - if len(found) != 3: - print("") - print("Your setup.py appears to be missing some important items") - print("(but I might be wrong). Please make sure it has something") - print("roughly like the following:") - print("") - print(" import versioneer") - print(" setup( version=versioneer.get_version(),") - print(" cmdclass=versioneer.get_cmdclass(), ...)") - print("") - errors += 1 - if setters: - print("You should remove lines like 'versioneer.VCS = ' and") - print("'versioneer.versionfile_source = ' . This configuration") - print("now lives in setup.cfg, and should be removed from setup.py") - print("") - errors += 1 - return errors - - -if __name__ == "__main__": - cmd = sys.argv[1] - if cmd == "setup": - errors = do_setup() - errors += scan_setup_py() - if errors: - sys.exit(1) diff --git a/python/cudf_kafka/cudf_kafka/_version.py b/python/cudf_kafka/cudf_kafka/_version.py deleted file mode 100644 index 3c1d113fd47..00000000000 --- a/python/cudf_kafka/cudf_kafka/_version.py +++ /dev/null @@ -1,566 +0,0 @@ -# Copyright (c) 2020-2022, NVIDIA CORPORATION. -# This file helps to compute a version number in source trees obtained from -# git-archive tarball (such as those provided by githubs download-from-tag -# feature). Distribution tarballs (built by setup.py sdist) and build -# directories (produced by setup.py build) will contain a much shorter file -# that just contains the computed version number. - -# This file is released into the public domain. Generated by -# versioneer-0.18 (https://github.com/warner/python-versioneer) - -"""Git implementation of _version.py.""" - -import errno -import os -import re -import subprocess -import sys - - -def get_keywords(): - """Get the keywords needed to look up the version information.""" - # these strings will be replaced by git during git-archive. - # setup.py/versioneer.py will grep for the variable names, so they must - # each be defined on a line of their own. _version.py will just call - # get_keywords(). - git_refnames = "$Format:%d$" - git_full = "$Format:%H$" - git_date = "$Format:%ci$" - keywords = {"refnames": git_refnames, "full": git_full, "date": git_date} - return keywords - - -class VersioneerConfig: - """Container for Versioneer configuration parameters.""" - - -def get_config(): - """Create, populate and return the VersioneerConfig() object.""" - # these strings are filled in when 'setup.py versioneer' creates - # _version.py - cfg = VersioneerConfig() - cfg.VCS = "git" - cfg.style = "pep440" - cfg.tag_prefix = "v" - cfg.parentdir_prefix = "cudf_kafka-" - cfg.versionfile_source = "cudf_kafka/_version.py" - cfg.verbose = False - return cfg - - -class NotThisMethod(Exception): - """Exception raised if a method is not valid for the current scenario.""" - - -LONG_VERSION_PY = {} -HANDLERS = {} - - -def register_vcs_handler(vcs, method): # decorator - """Decorator to mark a method as the handler for a particular VCS.""" - - def decorate(f): - """Store f in HANDLERS[vcs][method].""" - if vcs not in HANDLERS: - HANDLERS[vcs] = {} - HANDLERS[vcs][method] = f - return f - - return decorate - - -def run_command( - commands, args, cwd=None, verbose=False, hide_stderr=False, env=None -): - """Call the given command(s).""" - assert isinstance(commands, list) - p = None - for c in commands: - try: - dispcmd = str([c] + args) - # remember shell=False, so use git.cmd on windows, not just git - p = subprocess.Popen( - [c] + args, - cwd=cwd, - env=env, - stdout=subprocess.PIPE, - stderr=(subprocess.PIPE if hide_stderr else None), - ) - break - except OSError: - e = sys.exc_info()[1] - if e.errno == errno.ENOENT: - continue - if verbose: - print("unable to run %s" % dispcmd) - print(e) - return None, None - else: - if verbose: - print(f"unable to find command, tried {commands}") - return None, None - stdout = p.communicate()[0].strip() - if sys.version_info[0] >= 3: - stdout = stdout.decode() - if p.returncode != 0: - if verbose: - print("unable to run %s (error)" % dispcmd) - print("stdout was %s" % stdout) - return None, p.returncode - return stdout, p.returncode - - -def versions_from_parentdir(parentdir_prefix, root, verbose): - """Try to determine the version from the parent directory name. - - Source tarballs conventionally unpack into a directory that includes both - the project name and a version string. We will also support searching up - two directory levels for an appropriately named parent directory - """ - rootdirs = [] - - for i in range(3): - dirname = os.path.basename(root) - if dirname.startswith(parentdir_prefix): - return { - "version": dirname[len(parentdir_prefix) :], - "full-revisionid": None, - "dirty": False, - "error": None, - "date": None, - } - else: - rootdirs.append(root) - root = os.path.dirname(root) # up a level - - if verbose: - print( - "Tried directories %s but none started with prefix %s" - % (str(rootdirs), parentdir_prefix) - ) - raise NotThisMethod("rootdir doesn't start with parentdir_prefix") - - -@register_vcs_handler("git", "get_keywords") -def git_get_keywords(versionfile_abs): - """Extract version information from the given file.""" - # the code embedded in _version.py can just fetch the value of these - # keywords. When used from setup.py, we don't want to import _version.py, - # so we do it with a regexp instead. This function is not used from - # _version.py. - keywords = {} - try: - f = open(versionfile_abs) - for line in f.readlines(): - if line.strip().startswith("git_refnames ="): - mo = re.search(r'=\s*"(.*)"', line) - if mo: - keywords["refnames"] = mo.group(1) - if line.strip().startswith("git_full ="): - mo = re.search(r'=\s*"(.*)"', line) - if mo: - keywords["full"] = mo.group(1) - if line.strip().startswith("git_date ="): - mo = re.search(r'=\s*"(.*)"', line) - if mo: - keywords["date"] = mo.group(1) - f.close() - except OSError: - pass - return keywords - - -@register_vcs_handler("git", "keywords") -def git_versions_from_keywords(keywords, tag_prefix, verbose): - """Get version information from git keywords.""" - if not keywords: - raise NotThisMethod("no keywords at all, weird") - date = keywords.get("date") - if date is not None: - # git-2.2.0 added "%cI", which expands to an ISO-8601 -compliant - # datestamp. However we prefer "%ci" (which expands to an "ISO-8601 - # -like" string, which we must then edit to make compliant), because - # it's been around since git-1.5.3, and it's too difficult to - # discover which version we're using, or to work around using an - # older one. - date = date.strip().replace(" ", "T", 1).replace(" ", "", 1) - refnames = keywords["refnames"].strip() - if refnames.startswith("$Format"): - if verbose: - print("keywords are unexpanded, not using") - raise NotThisMethod("unexpanded keywords, not a git-archive tarball") - refs = {r.strip() for r in refnames.strip("()").split(",")} - # starting in git-1.8.3, tags are listed as "tag: foo-1.0" instead of - # just "foo-1.0". If we see a "tag: " prefix, prefer those. - TAG = "tag: " - tags = {r[len(TAG) :] for r in refs if r.startswith(TAG)} - if not tags: - # Either we're using git < 1.8.3, or there really are no tags. We use - # a heuristic: assume all version tags have a digit. The old git %d - # expansion behaves like git log --decorate=short and strips out the - # refs/heads/ and refs/tags/ prefixes that would let us distinguish - # between branches and tags. By ignoring refnames without digits, we - # filter out many common branch names like "release" and - # "stabilization", as well as "HEAD" and "master". - tags = {r for r in refs if re.search(r"\d", r)} - if verbose: - print("discarding '%s', no digits" % ",".join(refs - tags)) - if verbose: - print("likely tags: %s" % ",".join(sorted(tags))) - for ref in sorted(tags): - # sorting will prefer e.g. "2.0" over "2.0rc1" - if ref.startswith(tag_prefix): - r = ref[len(tag_prefix) :] - if verbose: - print("picking %s" % r) - return { - "version": r, - "full-revisionid": keywords["full"].strip(), - "dirty": False, - "error": None, - "date": date, - } - # no suitable tags, so version is "0+unknown", but full hex is still there - if verbose: - print("no suitable tags, using unknown + full revision id") - return { - "version": "0+unknown", - "full-revisionid": keywords["full"].strip(), - "dirty": False, - "error": "no suitable tags", - "date": None, - } - - -@register_vcs_handler("git", "pieces_from_vcs") -def git_pieces_from_vcs(tag_prefix, root, verbose, run_command=run_command): - """Get version from 'git describe' in the root of the source tree. - - This only gets called if the git-archive 'subst' keywords were *not* - expanded, and _version.py hasn't already been rewritten with a short - version string, meaning we're inside a checked out source tree. - """ - GITS = ["git"] - if sys.platform == "win32": - GITS = ["git.cmd", "git.exe"] - - out, rc = run_command( - GITS, ["rev-parse", "--git-dir"], cwd=root, hide_stderr=True - ) - if rc != 0: - if verbose: - print("Directory %s not under git control" % root) - raise NotThisMethod("'git rev-parse --git-dir' returned error") - - # if there is a tag matching tag_prefix, this yields TAG-NUM-gHEX[-dirty] - # if there isn't one, this yields HEX[-dirty] (no NUM) - describe_out, rc = run_command( - GITS, - [ - "describe", - "--tags", - "--dirty", - "--always", - "--long", - "--match", - "%s*" % tag_prefix, - ], - cwd=root, - ) - # --long was added in git-1.5.5 - if describe_out is None: - raise NotThisMethod("'git describe' failed") - describe_out = describe_out.strip() - full_out, rc = run_command(GITS, ["rev-parse", "HEAD"], cwd=root) - if full_out is None: - raise NotThisMethod("'git rev-parse' failed") - full_out = full_out.strip() - - pieces = {} - pieces["long"] = full_out - pieces["short"] = full_out[:7] # maybe improved later - pieces["error"] = None - - # parse describe_out. It will be like TAG-NUM-gHEX[-dirty] or HEX[-dirty] - # TAG might have hyphens. - git_describe = describe_out - - # look for -dirty suffix - dirty = git_describe.endswith("-dirty") - pieces["dirty"] = dirty - if dirty: - git_describe = git_describe[: git_describe.rindex("-dirty")] - - # now we have TAG-NUM-gHEX or HEX - - if "-" in git_describe: - # TAG-NUM-gHEX - mo = re.search(r"^(.+)-(\d+)-g([0-9a-f]+)$", git_describe) - if not mo: - # unparseable. Maybe git-describe is misbehaving? - pieces["error"] = ( - "unable to parse git-describe output: '%s'" % describe_out - ) - return pieces - - # tag - full_tag = mo.group(1) - if not full_tag.startswith(tag_prefix): - if verbose: - fmt = "tag '%s' doesn't start with prefix '%s'" - print(fmt % (full_tag, tag_prefix)) - pieces[ - "error" - ] = f"tag '{full_tag}' doesn't start with prefix '{tag_prefix}'" - return pieces - pieces["closest-tag"] = full_tag[len(tag_prefix) :] - - # distance: number of commits since tag - pieces["distance"] = int(mo.group(2)) - - # commit: short hex revision ID - pieces["short"] = mo.group(3) - - else: - # HEX: no tags - pieces["closest-tag"] = None - count_out, rc = run_command( - GITS, ["rev-list", "HEAD", "--count"], cwd=root - ) - pieces["distance"] = int(count_out) # total number of commits - - # commit date: see ISO-8601 comment in git_versions_from_keywords() - date = run_command(GITS, ["show", "-s", "--format=%ci", "HEAD"], cwd=root)[ - 0 - ].strip() - pieces["date"] = date.strip().replace(" ", "T", 1).replace(" ", "", 1) - - return pieces - - -def plus_or_dot(pieces): - """Return a + if we don't already have one, else return a .""" - if "+" in pieces.get("closest-tag", ""): - return "." - return "+" - - -def render_pep440(pieces): - """Build up version string, with post-release "local version identifier". - - Our goal: TAG[+DISTANCE.gHEX[.dirty]] . Note that if you - get a tagged build and then dirty it, you'll get TAG+0.gHEX.dirty - - Exceptions: - 1: no tags. git_describe was just HEX. 0+untagged.DISTANCE.gHEX[.dirty] - """ - if pieces["closest-tag"]: - rendered = pieces["closest-tag"] - if pieces["distance"] or pieces["dirty"]: - rendered += plus_or_dot(pieces) - rendered += "%d.g%s" % (pieces["distance"], pieces["short"]) - if pieces["dirty"]: - rendered += ".dirty" - else: - # exception #1 - rendered = "0+untagged.%d.g%s" % (pieces["distance"], pieces["short"]) - if pieces["dirty"]: - rendered += ".dirty" - return rendered - - -def render_pep440_pre(pieces): - """TAG[.post.devDISTANCE] -- No -dirty. - - Exceptions: - 1: no tags. 0.post.devDISTANCE - """ - if pieces["closest-tag"]: - rendered = pieces["closest-tag"] - if pieces["distance"]: - rendered += ".post.dev%d" % pieces["distance"] - else: - # exception #1 - rendered = "0.post.dev%d" % pieces["distance"] - return rendered - - -def render_pep440_post(pieces): - """TAG[.postDISTANCE[.dev0]+gHEX] . - - The ".dev0" means dirty. Note that .dev0 sorts backwards - (a dirty tree will appear "older" than the corresponding clean one), - but you shouldn't be releasing software with -dirty anyways. - - Exceptions: - 1: no tags. 0.postDISTANCE[.dev0] - """ - if pieces["closest-tag"]: - rendered = pieces["closest-tag"] - if pieces["distance"] or pieces["dirty"]: - rendered += ".post%d" % pieces["distance"] - if pieces["dirty"]: - rendered += ".dev0" - rendered += plus_or_dot(pieces) - rendered += "g%s" % pieces["short"] - else: - # exception #1 - rendered = "0.post%d" % pieces["distance"] - if pieces["dirty"]: - rendered += ".dev0" - rendered += "+g%s" % pieces["short"] - return rendered - - -def render_pep440_old(pieces): - """TAG[.postDISTANCE[.dev0]] . - - The ".dev0" means dirty. - - Exceptions: - 1: no tags. 0.postDISTANCE[.dev0] - """ - if pieces["closest-tag"]: - rendered = pieces["closest-tag"] - if pieces["distance"] or pieces["dirty"]: - rendered += ".post%d" % pieces["distance"] - if pieces["dirty"]: - rendered += ".dev0" - else: - # exception #1 - rendered = "0.post%d" % pieces["distance"] - if pieces["dirty"]: - rendered += ".dev0" - return rendered - - -def render_git_describe(pieces): - """TAG[-DISTANCE-gHEX][-dirty]. - - Like 'git describe --tags --dirty --always'. - - Exceptions: - 1: no tags. HEX[-dirty] (note: no 'g' prefix) - """ - if pieces["closest-tag"]: - rendered = pieces["closest-tag"] - if pieces["distance"]: - rendered += "-%d-g%s" % (pieces["distance"], pieces["short"]) - else: - # exception #1 - rendered = pieces["short"] - if pieces["dirty"]: - rendered += "-dirty" - return rendered - - -def render_git_describe_long(pieces): - """TAG-DISTANCE-gHEX[-dirty]. - - Like 'git describe --tags --dirty --always -long'. - The distance/hash is unconditional. - - Exceptions: - 1: no tags. HEX[-dirty] (note: no 'g' prefix) - """ - if pieces["closest-tag"]: - rendered = pieces["closest-tag"] - rendered += "-%d-g%s" % (pieces["distance"], pieces["short"]) - else: - # exception #1 - rendered = pieces["short"] - if pieces["dirty"]: - rendered += "-dirty" - return rendered - - -def render(pieces, style): - """Render the given version pieces into the requested style.""" - if pieces["error"]: - return { - "version": "unknown", - "full-revisionid": pieces.get("long"), - "dirty": None, - "error": pieces["error"], - "date": None, - } - - if not style or style == "default": - style = "pep440" # the default - - if style == "pep440": - rendered = render_pep440(pieces) - elif style == "pep440-pre": - rendered = render_pep440_pre(pieces) - elif style == "pep440-post": - rendered = render_pep440_post(pieces) - elif style == "pep440-old": - rendered = render_pep440_old(pieces) - elif style == "git-describe": - rendered = render_git_describe(pieces) - elif style == "git-describe-long": - rendered = render_git_describe_long(pieces) - else: - raise ValueError("unknown style '%s'" % style) - - return { - "version": rendered, - "full-revisionid": pieces["long"], - "dirty": pieces["dirty"], - "error": None, - "date": pieces.get("date"), - } - - -def get_versions(): - """Get version information or return default if unable to do so.""" - # I am in _version.py, which lives at ROOT/VERSIONFILE_SOURCE. If we have - # __file__, we can work backwards from there to the root. Some - # py2exe/bbfreeze/non-CPython implementations don't do __file__, in which - # case we can only use expanded keywords. - - cfg = get_config() - verbose = cfg.verbose - - try: - return git_versions_from_keywords( - get_keywords(), cfg.tag_prefix, verbose - ) - except NotThisMethod: - pass - - try: - root = os.path.realpath(__file__) - # versionfile_source is the relative path from the top of the source - # tree (where the .git directory might live) to this file. Invert - # this to find the root from __file__. - for i in cfg.versionfile_source.split("/"): - root = os.path.dirname(root) - except NameError: - return { - "version": "0+unknown", - "full-revisionid": None, - "dirty": None, - "error": "unable to find root of source tree", - "date": None, - } - - try: - pieces = git_pieces_from_vcs(cfg.tag_prefix, root, verbose) - return render(pieces, cfg.style) - except NotThisMethod: - pass - - try: - if cfg.parentdir_prefix: - return versions_from_parentdir(cfg.parentdir_prefix, root, verbose) - except NotThisMethod: - pass - - return { - "version": "0+unknown", - "full-revisionid": None, - "dirty": None, - "error": "unable to compute version", - "date": None, - } diff --git a/python/cudf_kafka/setup.cfg b/python/cudf_kafka/setup.cfg index f884e67908b..ee0d783b184 100644 --- a/python/cudf_kafka/setup.cfg +++ b/python/cudf_kafka/setup.cfg @@ -1,12 +1,4 @@ -# Copyright (c) 2020-2022, NVIDIA CORPORATION. - -[versioneer] -VCS = git -style = pep440 -versionfile_source = cudf_kafka/_version.py -versionfile_build = cudf_kafka/_version.py -tag_prefix = v -parentdir_prefix = cudf_kafka- +# Copyright (c) 2020-2023, NVIDIA CORPORATION. [isort] line_length=79 diff --git a/python/cudf_kafka/setup.py b/python/cudf_kafka/setup.py index caadfcac8aa..c39b65cdb55 100644 --- a/python/cudf_kafka/setup.py +++ b/python/cudf_kafka/setup.py @@ -6,7 +6,6 @@ import numpy as np import pyarrow as pa -import versioneer from Cython.Build import cythonize from setuptools import find_packages, setup from setuptools.extension import Extension @@ -87,7 +86,7 @@ setup( name="cudf_kafka", - version=versioneer.get_version(), + version="23.04.00", description="cuDF Kafka Datasource", url="https://github.com/rapidsai/cudf", author="NVIDIA Corporation", @@ -116,7 +115,6 @@ find_packages(include=["cudf_kafka._lib*"]), ["*.pxd"], ), - cmdclass=versioneer.get_cmdclass(), install_requires=install_requires, extras_require=extras_require, zip_safe=False, diff --git a/python/cudf_kafka/versioneer.py b/python/cudf_kafka/versioneer.py deleted file mode 100644 index dbddb6e0fd0..00000000000 --- a/python/cudf_kafka/versioneer.py +++ /dev/null @@ -1,1904 +0,0 @@ -# Copyright (c) 2020-2022, NVIDIA CORPORATION. -# Version: 0.18 - -"""The Versioneer - like a rocketeer, but for versions. - -The Versioneer -============== - -* like a rocketeer, but for versions! -* https://github.com/warner/python-versioneer -* Brian Warner -* License: Public Domain -* Compatible With: python2.6, 2.7, 3.2, 3.3, 3.4, 3.5, 3.6, and pypy -* [![Latest Version] -(https://pypip.in/version/versioneer/badge.svg?style=flat) -](https://pypi.python.org/pypi/versioneer/) -* [![Build Status] -(https://travis-ci.org/warner/python-versioneer.png?branch=master) -](https://travis-ci.org/warner/python-versioneer) - -This is a tool for managing a recorded version number in distutils-based -python projects. The goal is to remove the tedious and error-prone "update -the embedded version string" step from your release process. Making a new -release should be as easy as recording a new tag in your version-control -system, and maybe making new tarballs. - - -## Quick Install - -* `pip install versioneer` to somewhere to your $PATH -* add a `[versioneer]` section to your setup.cfg (see below) -* run `versioneer install` in your source tree, commit the results - -## Version Identifiers - -Source trees come from a variety of places: - -* a version-control system checkout (mostly used by developers) -* a nightly tarball, produced by build automation -* a snapshot tarball, produced by a web-based VCS browser, like github's - "tarball from tag" feature -* a release tarball, produced by "setup.py sdist", distributed through PyPI - -Within each source tree, the version identifier (either a string or a number, -this tool is format-agnostic) can come from a variety of places: - -* ask the VCS tool itself, e.g. "git describe" (for checkouts), which knows - about recent "tags" and an absolute revision-id -* the name of the directory into which the tarball was unpacked -* an expanded VCS keyword ($Id$, etc) -* a `_version.py` created by some earlier build step - -For released software, the version identifier is closely related to a VCS -tag. Some projects use tag names that include more than just the version -string (e.g. "myproject-1.2" instead of just "1.2"), in which case the tool -needs to strip the tag prefix to extract the version identifier. For -unreleased software (between tags), the version identifier should provide -enough information to help developers recreate the same tree, while also -giving them an idea of roughly how old the tree is (after version 1.2, before -version 1.3). Many VCS systems can report a description that captures this, -for example `git describe --tags --dirty --always` reports things like -"0.7-1-g574ab98-dirty" to indicate that the checkout is one revision past the -0.7 tag, has a unique revision id of "574ab98", and is "dirty" (it has -uncommitted changes. - -The version identifier is used for multiple purposes: - -* to allow the module to self-identify its version: `myproject.__version__` -* to choose a name and prefix for a 'setup.py sdist' tarball - -## Theory of Operation - -Versioneer works by adding a special `_version.py` file into your source -tree, where your `__init__.py` can import it. This `_version.py` knows how to -dynamically ask the VCS tool for version information at import time. - -`_version.py` also contains `$Revision$` markers, and the installation -process marks `_version.py` to have this marker rewritten with a tag name -during the `git archive` command. As a result, generated tarballs will -contain enough information to get the proper version. - -To allow `setup.py` to compute a version too, a `versioneer.py` is added to -the top level of your source tree, next to `setup.py` and the `setup.cfg` -that configures it. This overrides several distutils/setuptools commands to -compute the version when invoked, and changes `setup.py build` and `setup.py -sdist` to replace `_version.py` with a small static file that contains just -the generated version data. - -## Installation - -See [INSTALL.md](./INSTALL.md) for detailed installation instructions. - -## Version-String Flavors - -Code which uses Versioneer can learn about its version string at runtime by -importing `_version` from your main `__init__.py` file and running the -`get_versions()` function. From the "outside" (e.g. in `setup.py`), you can -import the top-level `versioneer.py` and run `get_versions()`. - -Both functions return a dictionary with different flavors of version -information: - -* `['version']`: A condensed version string, rendered using the selected - style. This is the most commonly used value for the project's version - string. The default "pep440" style yields strings like `0.11`, - `0.11+2.g1076c97`, or `0.11+2.g1076c97.dirty`. See the "Styles" section - below for alternative styles. - -* `['full-revisionid']`: detailed revision identifier. For Git, this is the - full SHA1 commit id, e.g. "1076c978a8d3cfc70f408fe5974aa6c092c949ac". - -* `['date']`: Date and time of the latest `HEAD` commit. For Git, it is the - commit date in ISO 8601 format. This will be None if the date is not - available. - -* `['dirty']`: a boolean, True if the tree has uncommitted changes. Note that - this is only accurate if run in a VCS checkout, otherwise it is likely to - be False or None - -* `['error']`: if the version string could not be computed, this will be set - to a string describing the problem, otherwise it will be None. It may be - useful to throw an exception in setup.py if this is set, to avoid e.g. - creating tarballs with a version string of "unknown". - -Some variants are more useful than others. Including `full-revisionid` in a -bug report should allow developers to reconstruct the exact code being tested -(or indicate the presence of local changes that should be shared with the -developers). `version` is suitable for display in an "about" box or a CLI -`--version` output: it can be easily compared against release notes and lists -of bugs fixed in various releases. - -The installer adds the following text to your `__init__.py` to place a basic -version in `YOURPROJECT.__version__`: - - from cudf_kafka._version import get_versions - __version__ = get_versions()['version'] - del get_versions - -## Styles - -The setup.cfg `style=` configuration controls how the VCS information is -rendered into a version string. - -The default style, "pep440", produces a PEP440-compliant string, equal to the -un-prefixed tag name for actual releases, and containing an additional "local -version" section with more detail for in-between builds. For Git, this is -TAG[+DISTANCE.gHEX[.dirty]] , using information from `git describe --tags ---dirty --always`. For example "0.11+2.g1076c97.dirty" indicates that the -tree is like the "1076c97" commit but has uncommitted changes (".dirty"), and -that this commit is two revisions ("+2") beyond the "0.11" tag. For released -software (exactly equal to a known tag), the identifier will only contain the -stripped tag, e.g. "0.11". - -Other styles are available. See [details.md](details.md) in the Versioneer -source tree for descriptions. - -## Debugging - -Versioneer tries to avoid fatal errors: if something goes wrong, it will tend -to return a version of "0+unknown". To investigate the problem, run `setup.py -version`, which will run the version-lookup code in a verbose mode, and will -display the full contents of `get_versions()` (including the `error` string, -which may help identify what went wrong). - -## Known Limitations - -Some situations are known to cause problems for Versioneer. This details the -most significant ones. More can be found on Github -[issues page](https://github.com/warner/python-versioneer/issues). - -### Subprojects - -Versioneer has limited support for source trees in which `setup.py` is not in -the root directory (e.g. `setup.py` and `.git/` are *not* siblings). The are -two common reasons why `setup.py` might not be in the root: - -* Source trees which contain multiple subprojects, such as - [Buildbot](https://github.com/buildbot/buildbot), which contains both - "master" and "slave" subprojects, each with their own `setup.py`, - `setup.cfg`, and `tox.ini`. Projects like these produce multiple PyPI - distributions (and upload multiple independently-installable tarballs). -* Source trees whose main purpose is to contain a C library, but which also - provide bindings to Python (and perhaps other langauges) in subdirectories. - -Versioneer will look for `.git` in parent directories, and most operations -should get the right version string. However `pip` and `setuptools` have bugs -and implementation details which frequently cause `pip install .` from a -subproject directory to fail to find a correct version string (so it usually -defaults to `0+unknown`). - -`pip install --editable .` should work correctly. `setup.py install` might -work too. - -Pip-8.1.1 is known to have this problem, but hopefully it will get fixed in -some later version. - -[Bug #38](https://github.com/warner/python-versioneer/issues/38) is tracking -this issue. The discussion in -[PR #61](https://github.com/warner/python-versioneer/pull/61) describes the -issue from the Versioneer side in more detail. -[pip PR#3176](https://github.com/pypa/pip/pull/3176) and -[pip PR#3615](https://github.com/pypa/pip/pull/3615) contain work to improve -pip to let Versioneer work correctly. - -Versioneer-0.16 and earlier only looked for a `.git` directory next to the -`setup.cfg`, so subprojects were completely unsupported with those releases. - -### Editable installs with setuptools <= 18.5 - -`setup.py develop` and `pip install --editable .` allow you to install a -project into a virtualenv once, then continue editing the source code (and -test) without re-installing after every change. - -"Entry-point scripts" (`setup(entry_points={"console_scripts": ..})`) are a -convenient way to specify executable scripts that should be installed along -with the python package. - -These both work as expected when using modern setuptools. When using -setuptools-18.5 or earlier, however, certain operations will cause -`pkg_resources.DistributionNotFound` errors when running the entrypoint -script, which must be resolved by re-installing the package. This happens -when the install happens with one version, then the egg_info data is -regenerated while a different version is checked out. Many setup.py commands -cause egg_info to be rebuilt (including `sdist`, `wheel`, and installing into -a different virtualenv), so this can be surprising. - -[Bug #83](https://github.com/warner/python-versioneer/issues/83) describes -this one, but upgrading to a newer version of setuptools should probably -resolve it. - -### Unicode version strings - -While Versioneer works (and is continually tested) with both Python 2 and -Python 3, it is not entirely consistent with bytes-vs-unicode distinctions. -Newer releases probably generate unicode version strings on py2. It's not -clear that this is wrong, but it may be surprising for applications when then -write these strings to a network connection or include them in bytes-oriented -APIs like cryptographic checksums. - -[Bug #71](https://github.com/warner/python-versioneer/issues/71) investigates -this question. - - -## Updating Versioneer - -To upgrade your project to a new release of Versioneer, do the following: - -* install the new Versioneer (`pip install -U versioneer` or equivalent) -* edit `setup.cfg`, if necessary, to include any new configuration settings - indicated by the release notes. See [UPGRADING](./UPGRADING.md) for details. -* re-run `versioneer install` in your source tree, to replace - `SRC/_version.py` -* commit any changed files - -## Future Directions - -This tool is designed to make it easily extended to other version-control -systems: all VCS-specific components are in separate directories like -src/git/ . The top-level `versioneer.py` script is assembled from these -components by running make-versioneer.py . In the future, make-versioneer.py -will take a VCS name as an argument, and will construct a version of -`versioneer.py` that is specific to the given VCS. It might also take the -configuration arguments that are currently provided manually during -installation by editing setup.py . Alternatively, it might go the other -direction and include code from all supported VCS systems, reducing the -number of intermediate scripts. - - -## License - -To make Versioneer easier to embed, all its code is dedicated to the public -domain. The `_version.py` that it creates is also in the public domain. -Specifically, both are released under the Creative Commons "Public Domain -Dedication" license (CC0-1.0), as described in -https://creativecommons.org/publicdomain/zero/1.0/ . - -""" - - -import errno -import json -import os -import re -import subprocess -import sys - -try: - import configparser -except ImportError: - import ConfigParser as configparser - - -class VersioneerConfig: - """Container for Versioneer configuration parameters.""" - - -def get_root(): - """Get the project root directory. - - We require that all commands are run from the project root, i.e. the - directory that contains setup.py, setup.cfg, and versioneer.py . - """ - root = os.path.realpath(os.path.abspath(os.getcwd())) - setup_py = os.path.join(root, "setup.py") - versioneer_py = os.path.join(root, "versioneer.py") - if not (os.path.exists(setup_py) or os.path.exists(versioneer_py)): - # allow 'python path/to/setup.py COMMAND' - root = os.path.dirname(os.path.realpath(os.path.abspath(sys.argv[0]))) - setup_py = os.path.join(root, "setup.py") - versioneer_py = os.path.join(root, "versioneer.py") - if not (os.path.exists(setup_py) or os.path.exists(versioneer_py)): - err = ( - "Versioneer was unable to run the project root directory. " - "Versioneer requires setup.py to be executed from " - "its immediate directory (like 'python setup.py COMMAND'), " - "or in a way that lets it use sys.argv[0] to find the root " - "(like 'python path/to/setup.py COMMAND')." - ) - raise VersioneerBadRootError(err) - try: - # Certain runtime workflows (setup.py install/develop in a setuptools - # tree) execute all dependencies in a single python process, so - # "versioneer" may be imported multiple times, and python's shared - # module-import table will cache the first one. So we can't use - # os.path.dirname(__file__), as that will find whichever - # versioneer.py was first imported, even in later projects. - me = os.path.realpath(os.path.abspath(__file__)) - me_dir = os.path.normcase(os.path.splitext(me)[0]) - vsr_dir = os.path.normcase(os.path.splitext(versioneer_py)[0]) - if me_dir != vsr_dir: - print( - "Warning: build in %s is using versioneer.py from %s" - % (os.path.dirname(me), versioneer_py) - ) - except NameError: - pass - return root - - -def get_config_from_root(root): - """Read the project setup.cfg file to determine Versioneer config.""" - # This might raise EnvironmentError (if setup.cfg is missing), or - # configparser.NoSectionError (if it lacks a [versioneer] section), or - # configparser.NoOptionError (if it lacks "VCS="). See the docstring at - # the top of versioneer.py for instructions on writing your setup.cfg . - setup_cfg = os.path.join(root, "setup.cfg") - parser = configparser.SafeConfigParser() - with open(setup_cfg) as f: - parser.readfp(f) - VCS = parser.get("versioneer", "VCS") # mandatory - - def get(parser, name): - if parser.has_option("versioneer", name): - return parser.get("versioneer", name) - return None - - cfg = VersioneerConfig() - cfg.VCS = VCS - cfg.style = get(parser, "style") or "" - cfg.versionfile_source = get(parser, "versionfile_source") - cfg.versionfile_build = get(parser, "versionfile_build") - cfg.tag_prefix = get(parser, "tag_prefix") - if cfg.tag_prefix in ("''", '""'): - cfg.tag_prefix = "" - cfg.parentdir_prefix = get(parser, "parentdir_prefix") - cfg.verbose = get(parser, "verbose") - return cfg - - -class NotThisMethod(Exception): - """Exception raised if a method is not valid for the current scenario.""" - - -# these dictionaries contain VCS-specific tools -LONG_VERSION_PY = {} -HANDLERS = {} - - -def register_vcs_handler(vcs, method): # decorator - """Decorator to mark a method as the handler for a particular VCS.""" - - def decorate(f): - """Store f in HANDLERS[vcs][method].""" - if vcs not in HANDLERS: - HANDLERS[vcs] = {} - HANDLERS[vcs][method] = f - return f - - return decorate - - -def run_command( - commands, args, cwd=None, verbose=False, hide_stderr=False, env=None -): - """Call the given command(s).""" - assert isinstance(commands, list) - p = None - for c in commands: - try: - dispcmd = str([c] + args) - # remember shell=False, so use git.cmd on windows, not just git - p = subprocess.Popen( - [c] + args, - cwd=cwd, - env=env, - stdout=subprocess.PIPE, - stderr=(subprocess.PIPE if hide_stderr else None), - ) - break - except OSError: - e = sys.exc_info()[1] - if e.errno == errno.ENOENT: - continue - if verbose: - print("unable to run %s" % dispcmd) - print(e) - return None, None - else: - if verbose: - print(f"unable to find command, tried {commands}") - return None, None - stdout = p.communicate()[0].strip() - if sys.version_info[0] >= 3: - stdout = stdout.decode() - if p.returncode != 0: - if verbose: - print("unable to run %s (error)" % dispcmd) - print("stdout was %s" % stdout) - return None, p.returncode - return stdout, p.returncode - - -LONG_VERSION_PY[ - "git" -] = r''' -# This file helps to compute a version number in source trees obtained from -# git-archive tarball (such as those provided by githubs download-from-tag -# feature). Distribution tarballs (built by setup.py sdist) and build -# directories (produced by setup.py build) will contain a much shorter file -# that just contains the computed version number. - -# This file is released into the public domain. Generated by -# versioneer-0.18 (https://github.com/warner/python-versioneer) - -"""Git implementation of _version.py.""" - -import errno -import os -import re -import subprocess -import sys - - -def get_keywords(): - """Get the keywords needed to look up the version information.""" - # these strings will be replaced by git during git-archive. - # setup.py/versioneer.py will grep for the variable names, so they must - # each be defined on a line of their own. _version.py will just call - # get_keywords(). - git_refnames = "%(DOLLAR)sFormat:%%d%(DOLLAR)s" - git_full = "%(DOLLAR)sFormat:%%H%(DOLLAR)s" - git_date = "%(DOLLAR)sFormat:%%ci%(DOLLAR)s" - keywords = {"refnames": git_refnames, "full": git_full, "date": git_date} - return keywords - - -class VersioneerConfig: - """Container for Versioneer configuration parameters.""" - - -def get_config(): - """Create, populate and return the VersioneerConfig() object.""" - # these strings are filled in when 'setup.py versioneer' creates - # _version.py - cfg = VersioneerConfig() - cfg.VCS = "git" - cfg.style = "%(STYLE)s" - cfg.tag_prefix = "%(TAG_PREFIX)s" - cfg.parentdir_prefix = "%(PARENTDIR_PREFIX)s" - cfg.versionfile_source = "%(VERSIONFILE_SOURCE)s" - cfg.verbose = False - return cfg - - -class NotThisMethod(Exception): - """Exception raised if a method is not valid for the current scenario.""" - - -LONG_VERSION_PY = {} -HANDLERS = {} - - -def register_vcs_handler(vcs, method): # decorator - """Decorator to mark a method as the handler for a particular VCS.""" - def decorate(f): - """Store f in HANDLERS[vcs][method].""" - if vcs not in HANDLERS: - HANDLERS[vcs] = {} - HANDLERS[vcs][method] = f - return f - return decorate - - -def run_command(commands, args, cwd=None, verbose=False, hide_stderr=False, - env=None): - """Call the given command(s).""" - assert isinstance(commands, list) - p = None - for c in commands: - try: - dispcmd = str([c] + args) - # remember shell=False, so use git.cmd on windows, not just git - p = subprocess.Popen([c] + args, cwd=cwd, env=env, - stdout=subprocess.PIPE, - stderr=(subprocess.PIPE if hide_stderr - else None)) - break - except EnvironmentError: - e = sys.exc_info()[1] - if e.errno == errno.ENOENT: - continue - if verbose: - print("unable to run %%s" %% dispcmd) - print(e) - return None, None - else: - if verbose: - print("unable to find command, tried %%s" %% (commands,)) - return None, None - stdout = p.communicate()[0].strip() - if sys.version_info[0] >= 3: - stdout = stdout.decode() - if p.returncode != 0: - if verbose: - print("unable to run %%s (error)" %% dispcmd) - print("stdout was %%s" %% stdout) - return None, p.returncode - return stdout, p.returncode - - -def versions_from_parentdir(parentdir_prefix, root, verbose): - """Try to determine the version from the parent directory name. - - Source tarballs conventionally unpack into a directory that includes both - the project name and a version string. We will also support searching up - two directory levels for an appropriately named parent directory - """ - rootdirs = [] - - for i in range(3): - dirname = os.path.basename(root) - if dirname.startswith(parentdir_prefix): - return {"version": dirname[len(parentdir_prefix):], - "full-revisionid": None, - "dirty": False, "error": None, "date": None} - else: - rootdirs.append(root) - root = os.path.dirname(root) # up a level - - if verbose: - print("Tried directories %%s but none started with prefix %%s" %% - (str(rootdirs), parentdir_prefix)) - raise NotThisMethod("rootdir doesn't start with parentdir_prefix") - - -@register_vcs_handler("git", "get_keywords") -def git_get_keywords(versionfile_abs): - """Extract version information from the given file.""" - # the code embedded in _version.py can just fetch the value of these - # keywords. When used from setup.py, we don't want to import _version.py, - # so we do it with a regexp instead. This function is not used from - # _version.py. - keywords = {} - try: - f = open(versionfile_abs, "r") - for line in f.readlines(): - if line.strip().startswith("git_refnames ="): - mo = re.search(r'=\s*"(.*)"', line) - if mo: - keywords["refnames"] = mo.group(1) - if line.strip().startswith("git_full ="): - mo = re.search(r'=\s*"(.*)"', line) - if mo: - keywords["full"] = mo.group(1) - if line.strip().startswith("git_date ="): - mo = re.search(r'=\s*"(.*)"', line) - if mo: - keywords["date"] = mo.group(1) - f.close() - except EnvironmentError: - pass - return keywords - - -@register_vcs_handler("git", "keywords") -def git_versions_from_keywords(keywords, tag_prefix, verbose): - """Get version information from git keywords.""" - if not keywords: - raise NotThisMethod("no keywords at all, weird") - date = keywords.get("date") - if date is not None: - # git-2.2.0 added "%%cI", which expands to an ISO-8601 -compliant - # datestamp. However we prefer "%%ci" (which expands to an "ISO-8601 - # -like" string, which we must then edit to make compliant), because - # it's been around since git-1.5.3, and it's too difficult to - # discover which version we're using, or to work around using an - # older one. - date = date.strip().replace(" ", "T", 1).replace(" ", "", 1) - refnames = keywords["refnames"].strip() - if refnames.startswith("$Format"): - if verbose: - print("keywords are unexpanded, not using") - raise NotThisMethod("unexpanded keywords, not a git-archive tarball") - refs = set([r.strip() for r in refnames.strip("()").split(",")]) - # starting in git-1.8.3, tags are listed as "tag: foo-1.0" instead of - # just "foo-1.0". If we see a "tag: " prefix, prefer those. - TAG = "tag: " - tags = set([r[len(TAG):] for r in refs if r.startswith(TAG)]) - if not tags: - # Either we're using git < 1.8.3, or there really are no tags. We use - # a heuristic: assume all version tags have a digit. The old git %%d - # expansion behaves like git log --decorate=short and strips out the - # refs/heads/ and refs/tags/ prefixes that would let us distinguish - # between branches and tags. By ignoring refnames without digits, we - # filter out many common branch names like "release" and - # "stabilization", as well as "HEAD" and "master". - tags = set([r for r in refs if re.search(r'\d', r)]) - if verbose: - print("discarding '%%s', no digits" %% ",".join(refs - tags)) - if verbose: - print("likely tags: %%s" %% ",".join(sorted(tags))) - for ref in sorted(tags): - # sorting will prefer e.g. "2.0" over "2.0rc1" - if ref.startswith(tag_prefix): - r = ref[len(tag_prefix):] - if verbose: - print("picking %%s" %% r) - return {"version": r, - "full-revisionid": keywords["full"].strip(), - "dirty": False, "error": None, - "date": date} - # no suitable tags, so version is "0+unknown", but full hex is still there - if verbose: - print("no suitable tags, using unknown + full revision id") - return {"version": "0+unknown", - "full-revisionid": keywords["full"].strip(), - "dirty": False, "error": "no suitable tags", "date": None} - - -@register_vcs_handler("git", "pieces_from_vcs") -def git_pieces_from_vcs(tag_prefix, root, verbose, run_command=run_command): - """Get version from 'git describe' in the root of the source tree. - - This only gets called if the git-archive 'subst' keywords were *not* - expanded, and _version.py hasn't already been rewritten with a short - version string, meaning we're inside a checked out source tree. - """ - GITS = ["git"] - if sys.platform == "win32": - GITS = ["git.cmd", "git.exe"] - - out, rc = run_command(GITS, ["rev-parse", "--git-dir"], cwd=root, - hide_stderr=True) - if rc != 0: - if verbose: - print("Directory %%s not under git control" %% root) - raise NotThisMethod("'git rev-parse --git-dir' returned error") - - # if there is a tag matching tag_prefix, this yields TAG-NUM-gHEX[-dirty] - # if there isn't one, this yields HEX[-dirty] (no NUM) - describe_out, rc = run_command(GITS, ["describe", "--tags", "--dirty", - "--always", "--long", - "--match", "%%s*" %% tag_prefix], - cwd=root) - # --long was added in git-1.5.5 - if describe_out is None: - raise NotThisMethod("'git describe' failed") - describe_out = describe_out.strip() - full_out, rc = run_command(GITS, ["rev-parse", "HEAD"], cwd=root) - if full_out is None: - raise NotThisMethod("'git rev-parse' failed") - full_out = full_out.strip() - - pieces = {} - pieces["long"] = full_out - pieces["short"] = full_out[:7] # maybe improved later - pieces["error"] = None - - # parse describe_out. It will be like TAG-NUM-gHEX[-dirty] or HEX[-dirty] - # TAG might have hyphens. - git_describe = describe_out - - # look for -dirty suffix - dirty = git_describe.endswith("-dirty") - pieces["dirty"] = dirty - if dirty: - git_describe = git_describe[:git_describe.rindex("-dirty")] - - # now we have TAG-NUM-gHEX or HEX - - if "-" in git_describe: - # TAG-NUM-gHEX - mo = re.search(r'^(.+)-(\d+)-g([0-9a-f]+)$', git_describe) - if not mo: - # unparseable. Maybe git-describe is misbehaving? - pieces["error"] = ("unable to parse git-describe output: '%%s'" - %% describe_out) - return pieces - - # tag - full_tag = mo.group(1) - if not full_tag.startswith(tag_prefix): - if verbose: - fmt = "tag '%%s' doesn't start with prefix '%%s'" - print(fmt %% (full_tag, tag_prefix)) - pieces["error"] = ("tag '%%s' doesn't start with prefix '%%s'" - %% (full_tag, tag_prefix)) - return pieces - pieces["closest-tag"] = full_tag[len(tag_prefix):] - - # distance: number of commits since tag - pieces["distance"] = int(mo.group(2)) - - # commit: short hex revision ID - pieces["short"] = mo.group(3) - - else: - # HEX: no tags - pieces["closest-tag"] = None - count_out, rc = run_command(GITS, ["rev-list", "HEAD", "--count"], - cwd=root) - pieces["distance"] = int(count_out) # total number of commits - - # commit date: see ISO-8601 comment in git_versions_from_keywords() - date = run_command(GITS, ["show", "-s", "--format=%%ci", "HEAD"], - cwd=root)[0].strip() - pieces["date"] = date.strip().replace(" ", "T", 1).replace(" ", "", 1) - - return pieces - - -def plus_or_dot(pieces): - """Return a + if we don't already have one, else return a .""" - if "+" in pieces.get("closest-tag", ""): - return "." - return "+" - - -def render_pep440(pieces): - """Build up version string, with post-release "local version identifier". - - Our goal: TAG[+DISTANCE.gHEX[.dirty]] . Note that if you - get a tagged build and then dirty it, you'll get TAG+0.gHEX.dirty - - Exceptions: - 1: no tags. git_describe was just HEX. 0+untagged.DISTANCE.gHEX[.dirty] - """ - if pieces["closest-tag"]: - rendered = pieces["closest-tag"] - if pieces["distance"] or pieces["dirty"]: - rendered += plus_or_dot(pieces) - rendered += "%%d.g%%s" %% (pieces["distance"], pieces["short"]) - if pieces["dirty"]: - rendered += ".dirty" - else: - # exception #1 - rendered = "0+untagged.%%d.g%%s" %% (pieces["distance"], - pieces["short"]) - if pieces["dirty"]: - rendered += ".dirty" - return rendered - - -def render_pep440_pre(pieces): - """TAG[.post.devDISTANCE] -- No -dirty. - - Exceptions: - 1: no tags. 0.post.devDISTANCE - """ - if pieces["closest-tag"]: - rendered = pieces["closest-tag"] - if pieces["distance"]: - rendered += ".post.dev%%d" %% pieces["distance"] - else: - # exception #1 - rendered = "0.post.dev%%d" %% pieces["distance"] - return rendered - - -def render_pep440_post(pieces): - """TAG[.postDISTANCE[.dev0]+gHEX] . - - The ".dev0" means dirty. Note that .dev0 sorts backwards - (a dirty tree will appear "older" than the corresponding clean one), - but you shouldn't be releasing software with -dirty anyways. - - Exceptions: - 1: no tags. 0.postDISTANCE[.dev0] - """ - if pieces["closest-tag"]: - rendered = pieces["closest-tag"] - if pieces["distance"] or pieces["dirty"]: - rendered += ".post%%d" %% pieces["distance"] - if pieces["dirty"]: - rendered += ".dev0" - rendered += plus_or_dot(pieces) - rendered += "g%%s" %% pieces["short"] - else: - # exception #1 - rendered = "0.post%%d" %% pieces["distance"] - if pieces["dirty"]: - rendered += ".dev0" - rendered += "+g%%s" %% pieces["short"] - return rendered - - -def render_pep440_old(pieces): - """TAG[.postDISTANCE[.dev0]] . - - The ".dev0" means dirty. - - Exceptions: - 1: no tags. 0.postDISTANCE[.dev0] - """ - if pieces["closest-tag"]: - rendered = pieces["closest-tag"] - if pieces["distance"] or pieces["dirty"]: - rendered += ".post%%d" %% pieces["distance"] - if pieces["dirty"]: - rendered += ".dev0" - else: - # exception #1 - rendered = "0.post%%d" %% pieces["distance"] - if pieces["dirty"]: - rendered += ".dev0" - return rendered - - -def render_git_describe(pieces): - """TAG[-DISTANCE-gHEX][-dirty]. - - Like 'git describe --tags --dirty --always'. - - Exceptions: - 1: no tags. HEX[-dirty] (note: no 'g' prefix) - """ - if pieces["closest-tag"]: - rendered = pieces["closest-tag"] - if pieces["distance"]: - rendered += "-%%d-g%%s" %% (pieces["distance"], pieces["short"]) - else: - # exception #1 - rendered = pieces["short"] - if pieces["dirty"]: - rendered += "-dirty" - return rendered - - -def render_git_describe_long(pieces): - """TAG-DISTANCE-gHEX[-dirty]. - - Like 'git describe --tags --dirty --always -long'. - The distance/hash is unconditional. - - Exceptions: - 1: no tags. HEX[-dirty] (note: no 'g' prefix) - """ - if pieces["closest-tag"]: - rendered = pieces["closest-tag"] - rendered += "-%%d-g%%s" %% (pieces["distance"], pieces["short"]) - else: - # exception #1 - rendered = pieces["short"] - if pieces["dirty"]: - rendered += "-dirty" - return rendered - - -def render(pieces, style): - """Render the given version pieces into the requested style.""" - if pieces["error"]: - return {"version": "unknown", - "full-revisionid": pieces.get("long"), - "dirty": None, - "error": pieces["error"], - "date": None} - - if not style or style == "default": - style = "pep440" # the default - - if style == "pep440": - rendered = render_pep440(pieces) - elif style == "pep440-pre": - rendered = render_pep440_pre(pieces) - elif style == "pep440-post": - rendered = render_pep440_post(pieces) - elif style == "pep440-old": - rendered = render_pep440_old(pieces) - elif style == "git-describe": - rendered = render_git_describe(pieces) - elif style == "git-describe-long": - rendered = render_git_describe_long(pieces) - else: - raise ValueError("unknown style '%%s'" %% style) - - return {"version": rendered, "full-revisionid": pieces["long"], - "dirty": pieces["dirty"], "error": None, - "date": pieces.get("date")} - - -def get_versions(): - """Get version information or return default if unable to do so.""" - # I am in _version.py, which lives at ROOT/VERSIONFILE_SOURCE. If we have - # __file__, we can work backwards from there to the root. Some - # py2exe/bbfreeze/non-CPython implementations don't do __file__, in which - # case we can only use expanded keywords. - - cfg = get_config() - verbose = cfg.verbose - - try: - return git_versions_from_keywords(get_keywords(), cfg.tag_prefix, - verbose) - except NotThisMethod: - pass - - try: - root = os.path.realpath(__file__) - # versionfile_source is the relative path from the top of the source - # tree (where the .git directory might live) to this file. Invert - # this to find the root from __file__. - for i in cfg.versionfile_source.split('/'): - root = os.path.dirname(root) - except NameError: - return {"version": "0+unknown", "full-revisionid": None, - "dirty": None, - "error": "unable to find root of source tree", - "date": None} - - try: - pieces = git_pieces_from_vcs(cfg.tag_prefix, root, verbose) - return render(pieces, cfg.style) - except NotThisMethod: - pass - - try: - if cfg.parentdir_prefix: - return versions_from_parentdir(cfg.parentdir_prefix, root, verbose) - except NotThisMethod: - pass - - return {"version": "0+unknown", "full-revisionid": None, - "dirty": None, - "error": "unable to compute version", "date": None} -''' - - -@register_vcs_handler("git", "get_keywords") -def git_get_keywords(versionfile_abs): - """Extract version information from the given file.""" - # the code embedded in _version.py can just fetch the value of these - # keywords. When used from setup.py, we don't want to import _version.py, - # so we do it with a regexp instead. This function is not used from - # _version.py. - keywords = {} - try: - f = open(versionfile_abs) - for line in f.readlines(): - if line.strip().startswith("git_refnames ="): - mo = re.search(r'=\s*"(.*)"', line) - if mo: - keywords["refnames"] = mo.group(1) - if line.strip().startswith("git_full ="): - mo = re.search(r'=\s*"(.*)"', line) - if mo: - keywords["full"] = mo.group(1) - if line.strip().startswith("git_date ="): - mo = re.search(r'=\s*"(.*)"', line) - if mo: - keywords["date"] = mo.group(1) - f.close() - except OSError: - pass - return keywords - - -@register_vcs_handler("git", "keywords") -def git_versions_from_keywords(keywords, tag_prefix, verbose): - """Get version information from git keywords.""" - if not keywords: - raise NotThisMethod("no keywords at all, weird") - date = keywords.get("date") - if date is not None: - # git-2.2.0 added "%cI", which expands to an ISO-8601 -compliant - # datestamp. However we prefer "%ci" (which expands to an "ISO-8601 - # -like" string, which we must then edit to make compliant), because - # it's been around since git-1.5.3, and it's too difficult to - # discover which version we're using, or to work around using an - # older one. - date = date.strip().replace(" ", "T", 1).replace(" ", "", 1) - refnames = keywords["refnames"].strip() - if refnames.startswith("$Format"): - if verbose: - print("keywords are unexpanded, not using") - raise NotThisMethod("unexpanded keywords, not a git-archive tarball") - refs = {r.strip() for r in refnames.strip("()").split(",")} - # starting in git-1.8.3, tags are listed as "tag: foo-1.0" instead of - # just "foo-1.0". If we see a "tag: " prefix, prefer those. - TAG = "tag: " - tags = {r[len(TAG) :] for r in refs if r.startswith(TAG)} - if not tags: - # Either we're using git < 1.8.3, or there really are no tags. We use - # a heuristic: assume all version tags have a digit. The old git %d - # expansion behaves like git log --decorate=short and strips out the - # refs/heads/ and refs/tags/ prefixes that would let us distinguish - # between branches and tags. By ignoring refnames without digits, we - # filter out many common branch names like "release" and - # "stabilization", as well as "HEAD" and "master". - tags = {r for r in refs if re.search(r"\d", r)} - if verbose: - print("discarding '%s', no digits" % ",".join(refs - tags)) - if verbose: - print("likely tags: %s" % ",".join(sorted(tags))) - for ref in sorted(tags): - # sorting will prefer e.g. "2.0" over "2.0rc1" - if ref.startswith(tag_prefix): - r = ref[len(tag_prefix) :] - if verbose: - print("picking %s" % r) - return { - "version": r, - "full-revisionid": keywords["full"].strip(), - "dirty": False, - "error": None, - "date": date, - } - # no suitable tags, so version is "0+unknown", but full hex is still there - if verbose: - print("no suitable tags, using unknown + full revision id") - return { - "version": "0+unknown", - "full-revisionid": keywords["full"].strip(), - "dirty": False, - "error": "no suitable tags", - "date": None, - } - - -@register_vcs_handler("git", "pieces_from_vcs") -def git_pieces_from_vcs(tag_prefix, root, verbose, run_command=run_command): - """Get version from 'git describe' in the root of the source tree. - - This only gets called if the git-archive 'subst' keywords were *not* - expanded, and _version.py hasn't already been rewritten with a short - version string, meaning we're inside a checked out source tree. - """ - GITS = ["git"] - if sys.platform == "win32": - GITS = ["git.cmd", "git.exe"] - - out, rc = run_command( - GITS, ["rev-parse", "--git-dir"], cwd=root, hide_stderr=True - ) - if rc != 0: - if verbose: - print("Directory %s not under git control" % root) - raise NotThisMethod("'git rev-parse --git-dir' returned error") - - # if there is a tag matching tag_prefix, this yields TAG-NUM-gHEX[-dirty] - # if there isn't one, this yields HEX[-dirty] (no NUM) - describe_out, rc = run_command( - GITS, - [ - "describe", - "--tags", - "--dirty", - "--always", - "--long", - "--match", - "%s*" % tag_prefix, - ], - cwd=root, - ) - # --long was added in git-1.5.5 - if describe_out is None: - raise NotThisMethod("'git describe' failed") - describe_out = describe_out.strip() - full_out, rc = run_command(GITS, ["rev-parse", "HEAD"], cwd=root) - if full_out is None: - raise NotThisMethod("'git rev-parse' failed") - full_out = full_out.strip() - - pieces = {} - pieces["long"] = full_out - pieces["short"] = full_out[:7] # maybe improved later - pieces["error"] = None - - # parse describe_out. It will be like TAG-NUM-gHEX[-dirty] or HEX[-dirty] - # TAG might have hyphens. - git_describe = describe_out - - # look for -dirty suffix - dirty = git_describe.endswith("-dirty") - pieces["dirty"] = dirty - if dirty: - git_describe = git_describe[: git_describe.rindex("-dirty")] - - # now we have TAG-NUM-gHEX or HEX - - if "-" in git_describe: - # TAG-NUM-gHEX - mo = re.search(r"^(.+)-(\d+)-g([0-9a-f]+)$", git_describe) - if not mo: - # unparseable. Maybe git-describe is misbehaving? - pieces["error"] = ( - "unable to parse git-describe output: '%s'" % describe_out - ) - return pieces - - # tag - full_tag = mo.group(1) - if not full_tag.startswith(tag_prefix): - if verbose: - fmt = "tag '%s' doesn't start with prefix '%s'" - print(fmt % (full_tag, tag_prefix)) - pieces["error"] = "tag '{}' doesn't start with prefix '{}'".format( - full_tag, - tag_prefix, - ) - return pieces - pieces["closest-tag"] = full_tag[len(tag_prefix) :] - - # distance: number of commits since tag - pieces["distance"] = int(mo.group(2)) - - # commit: short hex revision ID - pieces["short"] = mo.group(3) - - else: - # HEX: no tags - pieces["closest-tag"] = None - count_out, rc = run_command( - GITS, ["rev-list", "HEAD", "--count"], cwd=root - ) - pieces["distance"] = int(count_out) # total number of commits - - # commit date: see ISO-8601 comment in git_versions_from_keywords() - date = run_command(GITS, ["show", "-s", "--format=%ci", "HEAD"], cwd=root)[ - 0 - ].strip() - pieces["date"] = date.strip().replace(" ", "T", 1).replace(" ", "", 1) - - return pieces - - -def do_vcs_install(manifest_in, versionfile_source, ipy): - """Git-specific installation logic for Versioneer. - - For Git, this means creating/changing .gitattributes to mark _version.py - for export-subst keyword substitution. - """ - GITS = ["git"] - if sys.platform == "win32": - GITS = ["git.cmd", "git.exe"] - files = [manifest_in, versionfile_source] - if ipy: - files.append(ipy) - try: - me = __file__ - if me.endswith(".pyc") or me.endswith(".pyo"): - me = os.path.splitext(me)[0] + ".py" - versioneer_file = os.path.relpath(me) - except NameError: - versioneer_file = "versioneer.py" - files.append(versioneer_file) - present = False - try: - f = open(".gitattributes") - for line in f.readlines(): - if line.strip().startswith(versionfile_source): - if "export-subst" in line.strip().split()[1:]: - present = True - f.close() - except OSError: - pass - if not present: - f = open(".gitattributes", "a+") - f.write("%s export-subst\n" % versionfile_source) - f.close() - files.append(".gitattributes") - run_command(GITS, ["add", "--"] + files) - - -def versions_from_parentdir(parentdir_prefix, root, verbose): - """Try to determine the version from the parent directory name. - - Source tarballs conventionally unpack into a directory that includes both - the project name and a version string. We will also support searching up - two directory levels for an appropriately named parent directory - """ - rootdirs = [] - - for i in range(3): - dirname = os.path.basename(root) - if dirname.startswith(parentdir_prefix): - return { - "version": dirname[len(parentdir_prefix) :], - "full-revisionid": None, - "dirty": False, - "error": None, - "date": None, - } - else: - rootdirs.append(root) - root = os.path.dirname(root) # up a level - - if verbose: - print( - "Tried directories %s but none started with prefix %s" - % (str(rootdirs), parentdir_prefix) - ) - raise NotThisMethod("rootdir doesn't start with parentdir_prefix") - - -SHORT_VERSION_PY = """ -# This file was generated by 'versioneer.py' (0.18) from -# revision-control system data, or from the parent directory name of an -# unpacked source archive. Distribution tarballs contain a pre-generated copy -# of this file. - -import json - -version_json = ''' -%s -''' # END VERSION_JSON - - -def get_versions(): - return json.loads(version_json) -""" - - -def versions_from_file(filename): - """Try to determine the version from _version.py if present.""" - try: - with open(filename) as f: - contents = f.read() - except OSError: - raise NotThisMethod("unable to read _version.py") - mo = re.search( - r"version_json = '''\n(.*)''' # END VERSION_JSON", - contents, - re.M | re.S, - ) - if not mo: - mo = re.search( - r"version_json = '''\r\n(.*)''' # END VERSION_JSON", - contents, - re.M | re.S, - ) - if not mo: - raise NotThisMethod("no version_json in _version.py") - return json.loads(mo.group(1)) - - -def write_to_version_file(filename, versions): - """Write the given version number to the given _version.py file.""" - os.unlink(filename) - contents = json.dumps( - versions, sort_keys=True, indent=1, separators=(",", ": ") - ) - with open(filename, "w") as f: - f.write(SHORT_VERSION_PY % contents) - - print("set {} to '{}'".format(filename, versions["version"])) - - -def plus_or_dot(pieces): - """Return a + if we don't already have one, else return a .""" - if "+" in pieces.get("closest-tag", ""): - return "." - return "+" - - -def render_pep440(pieces): - """Build up version string, with post-release "local version identifier". - - Our goal: TAG[+DISTANCE.gHEX[.dirty]] . Note that if you - get a tagged build and then dirty it, you'll get TAG+0.gHEX.dirty - - Exceptions: - 1: no tags. git_describe was just HEX. 0+untagged.DISTANCE.gHEX[.dirty] - """ - if pieces["closest-tag"]: - rendered = pieces["closest-tag"] - if pieces["distance"] or pieces["dirty"]: - rendered += plus_or_dot(pieces) - rendered += "%d.g%s" % (pieces["distance"], pieces["short"]) - if pieces["dirty"]: - rendered += ".dirty" - else: - # exception #1 - rendered = "0+untagged.%d.g%s" % (pieces["distance"], pieces["short"]) - if pieces["dirty"]: - rendered += ".dirty" - return rendered - - -def render_pep440_pre(pieces): - """TAG[.post.devDISTANCE] -- No -dirty. - - Exceptions: - 1: no tags. 0.post.devDISTANCE - """ - if pieces["closest-tag"]: - rendered = pieces["closest-tag"] - if pieces["distance"]: - rendered += ".post.dev%d" % pieces["distance"] - else: - # exception #1 - rendered = "0.post.dev%d" % pieces["distance"] - return rendered - - -def render_pep440_post(pieces): - """TAG[.postDISTANCE[.dev0]+gHEX] . - - The ".dev0" means dirty. Note that .dev0 sorts backwards - (a dirty tree will appear "older" than the corresponding clean one), - but you shouldn't be releasing software with -dirty anyways. - - Exceptions: - 1: no tags. 0.postDISTANCE[.dev0] - """ - if pieces["closest-tag"]: - rendered = pieces["closest-tag"] - if pieces["distance"] or pieces["dirty"]: - rendered += ".post%d" % pieces["distance"] - if pieces["dirty"]: - rendered += ".dev0" - rendered += plus_or_dot(pieces) - rendered += "g%s" % pieces["short"] - else: - # exception #1 - rendered = "0.post%d" % pieces["distance"] - if pieces["dirty"]: - rendered += ".dev0" - rendered += "+g%s" % pieces["short"] - return rendered - - -def render_pep440_old(pieces): - """TAG[.postDISTANCE[.dev0]] . - - The ".dev0" means dirty. - - Exceptions: - 1: no tags. 0.postDISTANCE[.dev0] - """ - if pieces["closest-tag"]: - rendered = pieces["closest-tag"] - if pieces["distance"] or pieces["dirty"]: - rendered += ".post%d" % pieces["distance"] - if pieces["dirty"]: - rendered += ".dev0" - else: - # exception #1 - rendered = "0.post%d" % pieces["distance"] - if pieces["dirty"]: - rendered += ".dev0" - return rendered - - -def render_git_describe(pieces): - """TAG[-DISTANCE-gHEX][-dirty]. - - Like 'git describe --tags --dirty --always'. - - Exceptions: - 1: no tags. HEX[-dirty] (note: no 'g' prefix) - """ - if pieces["closest-tag"]: - rendered = pieces["closest-tag"] - if pieces["distance"]: - rendered += "-%d-g%s" % (pieces["distance"], pieces["short"]) - else: - # exception #1 - rendered = pieces["short"] - if pieces["dirty"]: - rendered += "-dirty" - return rendered - - -def render_git_describe_long(pieces): - """TAG-DISTANCE-gHEX[-dirty]. - - Like 'git describe --tags --dirty --always -long'. - The distance/hash is unconditional. - - Exceptions: - 1: no tags. HEX[-dirty] (note: no 'g' prefix) - """ - if pieces["closest-tag"]: - rendered = pieces["closest-tag"] - rendered += "-%d-g%s" % (pieces["distance"], pieces["short"]) - else: - # exception #1 - rendered = pieces["short"] - if pieces["dirty"]: - rendered += "-dirty" - return rendered - - -def render(pieces, style): - """Render the given version pieces into the requested style.""" - if pieces["error"]: - return { - "version": "unknown", - "full-revisionid": pieces.get("long"), - "dirty": None, - "error": pieces["error"], - "date": None, - } - - if not style or style == "default": - style = "pep440" # the default - - if style == "pep440": - rendered = render_pep440(pieces) - elif style == "pep440-pre": - rendered = render_pep440_pre(pieces) - elif style == "pep440-post": - rendered = render_pep440_post(pieces) - elif style == "pep440-old": - rendered = render_pep440_old(pieces) - elif style == "git-describe": - rendered = render_git_describe(pieces) - elif style == "git-describe-long": - rendered = render_git_describe_long(pieces) - else: - raise ValueError("unknown style '%s'" % style) - - return { - "version": rendered, - "full-revisionid": pieces["long"], - "dirty": pieces["dirty"], - "error": None, - "date": pieces.get("date"), - } - - -class VersioneerBadRootError(Exception): - """The project root directory is unknown or missing key files.""" - - -def get_versions(verbose=False): - """Get the project version from whatever source is available. - - Returns dict with two keys: 'version' and 'full'. - """ - if "versioneer" in sys.modules: - # see the discussion in cmdclass.py:get_cmdclass() - del sys.modules["versioneer"] - - root = get_root() - cfg = get_config_from_root(root) - - assert cfg.VCS is not None, "please set [versioneer]VCS= in setup.cfg" - handlers = HANDLERS.get(cfg.VCS) - assert handlers, "unrecognized VCS '%s'" % cfg.VCS - verbose = verbose or cfg.verbose - assert ( - cfg.versionfile_source is not None - ), "please set versioneer.versionfile_source" - assert cfg.tag_prefix is not None, "please set versioneer.tag_prefix" - - versionfile_abs = os.path.join(root, cfg.versionfile_source) - - # extract version from first of: _version.py, VCS command (e.g. 'git - # describe'), parentdir. This is meant to work for developers using a - # source checkout, for users of a tarball created by 'setup.py sdist', - # and for users of a tarball/zipball created by 'git archive' or github's - # download-from-tag feature or the equivalent in other VCSes. - - get_keywords_f = handlers.get("get_keywords") - from_keywords_f = handlers.get("keywords") - if get_keywords_f and from_keywords_f: - try: - keywords = get_keywords_f(versionfile_abs) - ver = from_keywords_f(keywords, cfg.tag_prefix, verbose) - if verbose: - print("got version from expanded keyword %s" % ver) - return ver - except NotThisMethod: - pass - - try: - ver = versions_from_file(versionfile_abs) - if verbose: - print(f"got version from file {versionfile_abs} {ver}") - return ver - except NotThisMethod: - pass - - from_vcs_f = handlers.get("pieces_from_vcs") - if from_vcs_f: - try: - pieces = from_vcs_f(cfg.tag_prefix, root, verbose) - ver = render(pieces, cfg.style) - if verbose: - print("got version from VCS %s" % ver) - return ver - except NotThisMethod: - pass - - try: - if cfg.parentdir_prefix: - ver = versions_from_parentdir(cfg.parentdir_prefix, root, verbose) - if verbose: - print("got version from parentdir %s" % ver) - return ver - except NotThisMethod: - pass - - if verbose: - print("unable to compute version") - - return { - "version": "0+unknown", - "full-revisionid": None, - "dirty": None, - "error": "unable to compute version", - "date": None, - } - - -def get_version(): - """Get the short version string for this project.""" - return get_versions()["version"] - - -def get_cmdclass(): - """Get the custom setuptools/distutils subclasses used by Versioneer.""" - if "versioneer" in sys.modules: - del sys.modules["versioneer"] - # this fixes the "python setup.py develop" case (also 'install' and - # 'easy_install .'), in which subdependencies of the main project are - # built (using setup.py bdist_egg) in the same python process. Assume - # a main project A and a dependency B, which use different versions - # of Versioneer. A's setup.py imports A's Versioneer, leaving it in - # sys.modules by the time B's setup.py is executed, causing B to run - # with the wrong versioneer. Setuptools wraps the sub-dep builds in a - # sandbox that restores sys.modules to it's pre-build state, so the - # parent is protected against the child's "import versioneer". By - # removing ourselves from sys.modules here, before the child build - # happens, we protect the child from the parent's versioneer too. - # Also see https://github.com/warner/python-versioneer/issues/52 - - cmds = {} - - # we add "version" to both distutils and setuptools - from distutils.core import Command - - class cmd_version(Command): - description = "report generated version string" - user_options = [] - boolean_options = [] - - def initialize_options(self): - pass - - def finalize_options(self): - pass - - def run(self): - vers = get_versions(verbose=True) - print("Version: %s" % vers["version"]) - print(" full-revisionid: %s" % vers.get("full-revisionid")) - print(" dirty: %s" % vers.get("dirty")) - print(" date: %s" % vers.get("date")) - if vers["error"]: - print(" error: %s" % vers["error"]) - - cmds["version"] = cmd_version - - # we override "build_py" in both distutils and setuptools - # - # most invocation pathways end up running build_py: - # distutils/build -> build_py - # distutils/install -> distutils/build ->.. - # setuptools/bdist_wheel -> distutils/install ->.. - # setuptools/bdist_egg -> distutils/install_lib -> build_py - # setuptools/install -> bdist_egg ->.. - # setuptools/develop -> ? - # pip install: - # copies source tree to a tempdir before running egg_info/etc - # if .git isn't copied too, 'git describe' will fail - # then does setup.py bdist_wheel, or sometimes setup.py install - # setup.py egg_info -> ? - - # we override different "build_py" commands for both environments - if "setuptools" in sys.modules: - from setuptools.command.build_py import build_py as _build_py - else: - from distutils.command.build_py import build_py as _build_py - - class cmd_build_py(_build_py): - def run(self): - root = get_root() - cfg = get_config_from_root(root) - versions = get_versions() - _build_py.run(self) - # now locate _version.py in the new build/ directory and replace - # it with an updated value - if cfg.versionfile_build: - target_versionfile = os.path.join( - self.build_lib, cfg.versionfile_build - ) - print("UPDATING %s" % target_versionfile) - write_to_version_file(target_versionfile, versions) - - cmds["build_py"] = cmd_build_py - - if "cx_Freeze" in sys.modules: # cx_freeze enabled? - from cx_Freeze.dist import build_exe as _build_exe - - # nczeczulin reports that py2exe won't like the pep440-style string - # as FILEVERSION, but it can be used for PRODUCTVERSION, e.g. - # setup(console=[{ - # "version": versioneer.get_version().split("+", 1)[0], # FILEVERSION - # "product_version": versioneer.get_version(), - # ... - - class cmd_build_exe(_build_exe): - def run(self): - root = get_root() - cfg = get_config_from_root(root) - versions = get_versions() - target_versionfile = cfg.versionfile_source - print("UPDATING %s" % target_versionfile) - write_to_version_file(target_versionfile, versions) - - _build_exe.run(self) - os.unlink(target_versionfile) - with open(cfg.versionfile_source, "w") as f: - LONG = LONG_VERSION_PY[cfg.VCS] - f.write( - LONG - % { - "DOLLAR": "$", - "STYLE": cfg.style, - "TAG_PREFIX": cfg.tag_prefix, - "PARENTDIR_PREFIX": cfg.parentdir_prefix, - "VERSIONFILE_SOURCE": cfg.versionfile_source, - } - ) - - cmds["build_exe"] = cmd_build_exe - del cmds["build_py"] - - if "py2exe" in sys.modules: # py2exe enabled? - try: - from py2exe.distutils_buildexe import py2exe as _py2exe # py3 - except ImportError: - from py2exe.build_exe import py2exe as _py2exe # py2 - - class cmd_py2exe(_py2exe): - def run(self): - root = get_root() - cfg = get_config_from_root(root) - versions = get_versions() - target_versionfile = cfg.versionfile_source - print("UPDATING %s" % target_versionfile) - write_to_version_file(target_versionfile, versions) - - _py2exe.run(self) - os.unlink(target_versionfile) - with open(cfg.versionfile_source, "w") as f: - LONG = LONG_VERSION_PY[cfg.VCS] - f.write( - LONG - % { - "DOLLAR": "$", - "STYLE": cfg.style, - "TAG_PREFIX": cfg.tag_prefix, - "PARENTDIR_PREFIX": cfg.parentdir_prefix, - "VERSIONFILE_SOURCE": cfg.versionfile_source, - } - ) - - cmds["py2exe"] = cmd_py2exe - - # we override different "sdist" commands for both environments - if "setuptools" in sys.modules: - from setuptools.command.sdist import sdist as _sdist - else: - from distutils.command.sdist import sdist as _sdist - - class cmd_sdist(_sdist): - def run(self): - versions = get_versions() - self._versioneer_generated_versions = versions - # unless we update this, the command will keep using the old - # version - self.distribution.metadata.version = versions["version"] - return _sdist.run(self) - - def make_release_tree(self, base_dir, files): - root = get_root() - cfg = get_config_from_root(root) - _sdist.make_release_tree(self, base_dir, files) - # now locate _version.py in the new base_dir directory - # (remembering that it may be a hardlink) and replace it with an - # updated value - target_versionfile = os.path.join(base_dir, cfg.versionfile_source) - print("UPDATING %s" % target_versionfile) - write_to_version_file( - target_versionfile, self._versioneer_generated_versions - ) - - cmds["sdist"] = cmd_sdist - - return cmds - - -CONFIG_ERROR = """ -setup.cfg is missing the necessary Versioneer configuration. You need -a section like: - - [versioneer] - VCS = git - style = pep440 - versionfile_source = src/myproject/_version.py - versionfile_build = myproject/_version.py - tag_prefix = - parentdir_prefix = myproject- - -You will also need to edit your setup.py to use the results: - - import versioneer - setup(version=versioneer.get_version(), - cmdclass=versioneer.get_cmdclass(), ...) - -Please read the docstring in ./versioneer.py for configuration instructions, -edit setup.cfg, and re-run the installer or 'python versioneer.py setup'. -""" - -SAMPLE_CONFIG = """ -# See the docstring in versioneer.py for instructions. Note that you must -# re-run 'versioneer.py setup' after changing this section, and commit the -# resulting files. - -[versioneer] -#VCS = git -#style = pep440 -#versionfile_source = -#versionfile_build = -#tag_prefix = -#parentdir_prefix = - -""" - -INIT_PY_SNIPPET = """ -from cudf_kafka._version import get_versions -__version__ = get_versions()['version'] -del get_versions -""" - - -def do_setup(): - """Main VCS-independent setup function for installing Versioneer.""" - root = get_root() - try: - cfg = get_config_from_root(root) - except ( - OSError, - configparser.NoSectionError, - configparser.NoOptionError, - ) as e: - if isinstance(e, (EnvironmentError, configparser.NoSectionError)): - print( - "Adding sample versioneer config to setup.cfg", file=sys.stderr - ) - with open(os.path.join(root, "setup.cfg"), "a") as f: - f.write(SAMPLE_CONFIG) - print(CONFIG_ERROR, file=sys.stderr) - return 1 - - print(" creating %s" % cfg.versionfile_source) - with open(cfg.versionfile_source, "w") as f: - LONG = LONG_VERSION_PY[cfg.VCS] - f.write( - LONG - % { - "DOLLAR": "$", - "STYLE": cfg.style, - "TAG_PREFIX": cfg.tag_prefix, - "PARENTDIR_PREFIX": cfg.parentdir_prefix, - "VERSIONFILE_SOURCE": cfg.versionfile_source, - } - ) - - ipy = os.path.join(os.path.dirname(cfg.versionfile_source), "__init__.py") - if os.path.exists(ipy): - try: - with open(ipy) as f: - old = f.read() - except OSError: - old = "" - if INIT_PY_SNIPPET not in old: - print(" appending to %s" % ipy) - with open(ipy, "a") as f: - f.write(INIT_PY_SNIPPET) - else: - print(" %s unmodified" % ipy) - else: - print(" %s doesn't exist, ok" % ipy) - ipy = None - - # Make sure both the top-level "versioneer.py" and versionfile_source - # (PKG/_version.py, used by runtime code) are in MANIFEST.in, so - # they'll be copied into source distributions. Pip won't be able to - # install the package without this. - manifest_in = os.path.join(root, "MANIFEST.in") - simple_includes = set() - try: - with open(manifest_in) as f: - for line in f: - if line.startswith("include "): - for include in line.split()[1:]: - simple_includes.add(include) - except OSError: - pass - # That doesn't cover everything MANIFEST.in can do - # (http://docs.python.org/2/distutils/sourcedist.html#commands), so - # it might give some false negatives. Appending redundant 'include' - # lines is safe, though. - if "versioneer.py" not in simple_includes: - print(" appending 'versioneer.py' to MANIFEST.in") - with open(manifest_in, "a") as f: - f.write("include versioneer.py\n") - else: - print(" 'versioneer.py' already in MANIFEST.in") - if cfg.versionfile_source not in simple_includes: - print( - " appending versionfile_source ('%s') to MANIFEST.in" - % cfg.versionfile_source - ) - with open(manifest_in, "a") as f: - f.write("include %s\n" % cfg.versionfile_source) - else: - print(" versionfile_source already in MANIFEST.in") - - # Make VCS-specific changes. For git, this means creating/changing - # .gitattributes to mark _version.py for export-subst keyword - # substitution. - do_vcs_install(manifest_in, cfg.versionfile_source, ipy) - return 0 - - -def scan_setup_py(): - """Validate the contents of setup.py against Versioneer's expectations.""" - found = set() - setters = False - errors = 0 - with open("setup.py") as f: - for line in f.readlines(): - if "import versioneer" in line: - found.add("import") - if "versioneer.get_cmdclass()" in line: - found.add("cmdclass") - if "versioneer.get_version()" in line: - found.add("get_version") - if "versioneer.VCS" in line: - setters = True - if "versioneer.versionfile_source" in line: - setters = True - if len(found) != 3: - print("") - print("Your setup.py appears to be missing some important items") - print("(but I might be wrong). Please make sure it has something") - print("roughly like the following:") - print("") - print(" import versioneer") - print(" setup( version=versioneer.get_version(),") - print(" cmdclass=versioneer.get_cmdclass(), ...)") - print("") - errors += 1 - if setters: - print("You should remove lines like 'versioneer.VCS = ' and") - print("'versioneer.versionfile_source = ' . This configuration") - print("now lives in setup.cfg, and should be removed from setup.py") - print("") - errors += 1 - return errors - - -if __name__ == "__main__": - cmd = sys.argv[1] - if cmd == "setup": - errors = do_setup() - errors += scan_setup_py() - if errors: - sys.exit(1) diff --git a/python/custreamz/custreamz/_version.py b/python/custreamz/custreamz/_version.py deleted file mode 100644 index a017486df32..00000000000 --- a/python/custreamz/custreamz/_version.py +++ /dev/null @@ -1,566 +0,0 @@ -# Copyright (c) 2020-2022, NVIDIA CORPORATION. -# This file helps to compute a version number in source trees obtained from -# git-archive tarball (such as those provided by githubs download-from-tag -# feature). Distribution tarballs (built by setup.py sdist) and build -# directories (produced by setup.py build) will contain a much shorter file -# that just contains the computed version number. - -# This file is released into the public domain. Generated by -# versioneer-0.18 (https://github.com/warner/python-versioneer) - -"""Git implementation of _version.py.""" - -import errno -import os -import re -import subprocess -import sys - - -def get_keywords(): - """Get the keywords needed to look up the version information.""" - # these strings will be replaced by git during git-archive. - # setup.py/versioneer.py will grep for the variable names, so they must - # each be defined on a line of their own. _version.py will just call - # get_keywords(). - git_refnames = "$Format:%d$" - git_full = "$Format:%H$" - git_date = "$Format:%ci$" - keywords = {"refnames": git_refnames, "full": git_full, "date": git_date} - return keywords - - -class VersioneerConfig: - """Container for Versioneer configuration parameters.""" - - -def get_config(): - """Create, populate and return the VersioneerConfig() object.""" - # these strings are filled in when 'setup.py versioneer' creates - # _version.py - cfg = VersioneerConfig() - cfg.VCS = "git" - cfg.style = "pep440" - cfg.tag_prefix = "v" - cfg.parentdir_prefix = "custreamz-" - cfg.versionfile_source = "custreamz/_version.py" - cfg.verbose = False - return cfg - - -class NotThisMethod(Exception): - """Exception raised if a method is not valid for the current scenario.""" - - -LONG_VERSION_PY = {} -HANDLERS = {} - - -def register_vcs_handler(vcs, method): # decorator - """Decorator to mark a method as the handler for a particular VCS.""" - - def decorate(f): - """Store f in HANDLERS[vcs][method].""" - if vcs not in HANDLERS: - HANDLERS[vcs] = {} - HANDLERS[vcs][method] = f - return f - - return decorate - - -def run_command( - commands, args, cwd=None, verbose=False, hide_stderr=False, env=None -): - """Call the given command(s).""" - assert isinstance(commands, list) - p = None - for c in commands: - try: - dispcmd = str([c] + args) - # remember shell=False, so use git.cmd on windows, not just git - p = subprocess.Popen( - [c] + args, - cwd=cwd, - env=env, - stdout=subprocess.PIPE, - stderr=(subprocess.PIPE if hide_stderr else None), - ) - break - except OSError: - e = sys.exc_info()[1] - if e.errno == errno.ENOENT: - continue - if verbose: - print("unable to run %s" % dispcmd) - print(e) - return None, None - else: - if verbose: - print(f"unable to find command, tried {commands}") - return None, None - stdout = p.communicate()[0].strip() - if sys.version_info[0] >= 3: - stdout = stdout.decode() - if p.returncode != 0: - if verbose: - print("unable to run %s (error)" % dispcmd) - print("stdout was %s" % stdout) - return None, p.returncode - return stdout, p.returncode - - -def versions_from_parentdir(parentdir_prefix, root, verbose): - """Try to determine the version from the parent directory name. - - Source tarballs conventionally unpack into a directory that includes both - the project name and a version string. We will also support searching up - two directory levels for an appropriately named parent directory - """ - rootdirs = [] - - for i in range(3): - dirname = os.path.basename(root) - if dirname.startswith(parentdir_prefix): - return { - "version": dirname[len(parentdir_prefix) :], - "full-revisionid": None, - "dirty": False, - "error": None, - "date": None, - } - else: - rootdirs.append(root) - root = os.path.dirname(root) # up a level - - if verbose: - print( - "Tried directories %s but none started with prefix %s" - % (str(rootdirs), parentdir_prefix) - ) - raise NotThisMethod("rootdir doesn't start with parentdir_prefix") - - -@register_vcs_handler("git", "get_keywords") -def git_get_keywords(versionfile_abs): - """Extract version information from the given file.""" - # the code embedded in _version.py can just fetch the value of these - # keywords. When used from setup.py, we don't want to import _version.py, - # so we do it with a regexp instead. This function is not used from - # _version.py. - keywords = {} - try: - f = open(versionfile_abs) - for line in f.readlines(): - if line.strip().startswith("git_refnames ="): - mo = re.search(r'=\s*"(.*)"', line) - if mo: - keywords["refnames"] = mo.group(1) - if line.strip().startswith("git_full ="): - mo = re.search(r'=\s*"(.*)"', line) - if mo: - keywords["full"] = mo.group(1) - if line.strip().startswith("git_date ="): - mo = re.search(r'=\s*"(.*)"', line) - if mo: - keywords["date"] = mo.group(1) - f.close() - except OSError: - pass - return keywords - - -@register_vcs_handler("git", "keywords") -def git_versions_from_keywords(keywords, tag_prefix, verbose): - """Get version information from git keywords.""" - if not keywords: - raise NotThisMethod("no keywords at all, weird") - date = keywords.get("date") - if date is not None: - # git-2.2.0 added "%cI", which expands to an ISO-8601 -compliant - # datestamp. However we prefer "%ci" (which expands to an "ISO-8601 - # -like" string, which we must then edit to make compliant), because - # it's been around since git-1.5.3, and it's too difficult to - # discover which version we're using, or to work around using an - # older one. - date = date.strip().replace(" ", "T", 1).replace(" ", "", 1) - refnames = keywords["refnames"].strip() - if refnames.startswith("$Format"): - if verbose: - print("keywords are unexpanded, not using") - raise NotThisMethod("unexpanded keywords, not a git-archive tarball") - refs = {r.strip() for r in refnames.strip("()").split(",")} - # starting in git-1.8.3, tags are listed as "tag: foo-1.0" instead of - # just "foo-1.0". If we see a "tag: " prefix, prefer those. - TAG = "tag: " - tags = {r[len(TAG) :] for r in refs if r.startswith(TAG)} - if not tags: - # Either we're using git < 1.8.3, or there really are no tags. We use - # a heuristic: assume all version tags have a digit. The old git %d - # expansion behaves like git log --decorate=short and strips out the - # refs/heads/ and refs/tags/ prefixes that would let us distinguish - # between branches and tags. By ignoring refnames without digits, we - # filter out many common branch names like "release" and - # "stabilization", as well as "HEAD" and "master". - tags = {r for r in refs if re.search(r"\d", r)} - if verbose: - print("discarding '%s', no digits" % ",".join(refs - tags)) - if verbose: - print("likely tags: %s" % ",".join(sorted(tags))) - for ref in sorted(tags): - # sorting will prefer e.g. "2.0" over "2.0rc1" - if ref.startswith(tag_prefix): - r = ref[len(tag_prefix) :] - if verbose: - print("picking %s" % r) - return { - "version": r, - "full-revisionid": keywords["full"].strip(), - "dirty": False, - "error": None, - "date": date, - } - # no suitable tags, so version is "0+unknown", but full hex is still there - if verbose: - print("no suitable tags, using unknown + full revision id") - return { - "version": "0+unknown", - "full-revisionid": keywords["full"].strip(), - "dirty": False, - "error": "no suitable tags", - "date": None, - } - - -@register_vcs_handler("git", "pieces_from_vcs") -def git_pieces_from_vcs(tag_prefix, root, verbose, run_command=run_command): - """Get version from 'git describe' in the root of the source tree. - - This only gets called if the git-archive 'subst' keywords were *not* - expanded, and _version.py hasn't already been rewritten with a short - version string, meaning we're inside a checked out source tree. - """ - GITS = ["git"] - if sys.platform == "win32": - GITS = ["git.cmd", "git.exe"] - - out, rc = run_command( - GITS, ["rev-parse", "--git-dir"], cwd=root, hide_stderr=True - ) - if rc != 0: - if verbose: - print("Directory %s not under git control" % root) - raise NotThisMethod("'git rev-parse --git-dir' returned error") - - # if there is a tag matching tag_prefix, this yields TAG-NUM-gHEX[-dirty] - # if there isn't one, this yields HEX[-dirty] (no NUM) - describe_out, rc = run_command( - GITS, - [ - "describe", - "--tags", - "--dirty", - "--always", - "--long", - "--match", - "%s*" % tag_prefix, - ], - cwd=root, - ) - # --long was added in git-1.5.5 - if describe_out is None: - raise NotThisMethod("'git describe' failed") - describe_out = describe_out.strip() - full_out, rc = run_command(GITS, ["rev-parse", "HEAD"], cwd=root) - if full_out is None: - raise NotThisMethod("'git rev-parse' failed") - full_out = full_out.strip() - - pieces = {} - pieces["long"] = full_out - pieces["short"] = full_out[:7] # maybe improved later - pieces["error"] = None - - # parse describe_out. It will be like TAG-NUM-gHEX[-dirty] or HEX[-dirty] - # TAG might have hyphens. - git_describe = describe_out - - # look for -dirty suffix - dirty = git_describe.endswith("-dirty") - pieces["dirty"] = dirty - if dirty: - git_describe = git_describe[: git_describe.rindex("-dirty")] - - # now we have TAG-NUM-gHEX or HEX - - if "-" in git_describe: - # TAG-NUM-gHEX - mo = re.search(r"^(.+)-(\d+)-g([0-9a-f]+)$", git_describe) - if not mo: - # unparseable. Maybe git-describe is misbehaving? - pieces["error"] = ( - "unable to parse git-describe output: '%s'" % describe_out - ) - return pieces - - # tag - full_tag = mo.group(1) - if not full_tag.startswith(tag_prefix): - if verbose: - fmt = "tag '%s' doesn't start with prefix '%s'" - print(fmt % (full_tag, tag_prefix)) - pieces[ - "error" - ] = f"tag '{full_tag}' doesn't start with prefix '{tag_prefix}'" - return pieces - pieces["closest-tag"] = full_tag[len(tag_prefix) :] - - # distance: number of commits since tag - pieces["distance"] = int(mo.group(2)) - - # commit: short hex revision ID - pieces["short"] = mo.group(3) - - else: - # HEX: no tags - pieces["closest-tag"] = None - count_out, rc = run_command( - GITS, ["rev-list", "HEAD", "--count"], cwd=root - ) - pieces["distance"] = int(count_out) # total number of commits - - # commit date: see ISO-8601 comment in git_versions_from_keywords() - date = run_command(GITS, ["show", "-s", "--format=%ci", "HEAD"], cwd=root)[ - 0 - ].strip() - pieces["date"] = date.strip().replace(" ", "T", 1).replace(" ", "", 1) - - return pieces - - -def plus_or_dot(pieces): - """Return a + if we don't already have one, else return a .""" - if "+" in pieces.get("closest-tag", ""): - return "." - return "+" - - -def render_pep440(pieces): - """Build up version string, with post-release "local version identifier". - - Our goal: TAG[+DISTANCE.gHEX[.dirty]] . Note that if you - get a tagged build and then dirty it, you'll get TAG+0.gHEX.dirty - - Exceptions: - 1: no tags. git_describe was just HEX. 0+untagged.DISTANCE.gHEX[.dirty] - """ - if pieces["closest-tag"]: - rendered = pieces["closest-tag"] - if pieces["distance"] or pieces["dirty"]: - rendered += plus_or_dot(pieces) - rendered += "%d.g%s" % (pieces["distance"], pieces["short"]) - if pieces["dirty"]: - rendered += ".dirty" - else: - # exception #1 - rendered = "0+untagged.%d.g%s" % (pieces["distance"], pieces["short"]) - if pieces["dirty"]: - rendered += ".dirty" - return rendered - - -def render_pep440_pre(pieces): - """TAG[.post.devDISTANCE] -- No -dirty. - - Exceptions: - 1: no tags. 0.post.devDISTANCE - """ - if pieces["closest-tag"]: - rendered = pieces["closest-tag"] - if pieces["distance"]: - rendered += ".post.dev%d" % pieces["distance"] - else: - # exception #1 - rendered = "0.post.dev%d" % pieces["distance"] - return rendered - - -def render_pep440_post(pieces): - """TAG[.postDISTANCE[.dev0]+gHEX] . - - The ".dev0" means dirty. Note that .dev0 sorts backwards - (a dirty tree will appear "older" than the corresponding clean one), - but you shouldn't be releasing software with -dirty anyways. - - Exceptions: - 1: no tags. 0.postDISTANCE[.dev0] - """ - if pieces["closest-tag"]: - rendered = pieces["closest-tag"] - if pieces["distance"] or pieces["dirty"]: - rendered += ".post%d" % pieces["distance"] - if pieces["dirty"]: - rendered += ".dev0" - rendered += plus_or_dot(pieces) - rendered += "g%s" % pieces["short"] - else: - # exception #1 - rendered = "0.post%d" % pieces["distance"] - if pieces["dirty"]: - rendered += ".dev0" - rendered += "+g%s" % pieces["short"] - return rendered - - -def render_pep440_old(pieces): - """TAG[.postDISTANCE[.dev0]] . - - The ".dev0" means dirty. - - Exceptions: - 1: no tags. 0.postDISTANCE[.dev0] - """ - if pieces["closest-tag"]: - rendered = pieces["closest-tag"] - if pieces["distance"] or pieces["dirty"]: - rendered += ".post%d" % pieces["distance"] - if pieces["dirty"]: - rendered += ".dev0" - else: - # exception #1 - rendered = "0.post%d" % pieces["distance"] - if pieces["dirty"]: - rendered += ".dev0" - return rendered - - -def render_git_describe(pieces): - """TAG[-DISTANCE-gHEX][-dirty]. - - Like 'git describe --tags --dirty --always'. - - Exceptions: - 1: no tags. HEX[-dirty] (note: no 'g' prefix) - """ - if pieces["closest-tag"]: - rendered = pieces["closest-tag"] - if pieces["distance"]: - rendered += "-%d-g%s" % (pieces["distance"], pieces["short"]) - else: - # exception #1 - rendered = pieces["short"] - if pieces["dirty"]: - rendered += "-dirty" - return rendered - - -def render_git_describe_long(pieces): - """TAG-DISTANCE-gHEX[-dirty]. - - Like 'git describe --tags --dirty --always -long'. - The distance/hash is unconditional. - - Exceptions: - 1: no tags. HEX[-dirty] (note: no 'g' prefix) - """ - if pieces["closest-tag"]: - rendered = pieces["closest-tag"] - rendered += "-%d-g%s" % (pieces["distance"], pieces["short"]) - else: - # exception #1 - rendered = pieces["short"] - if pieces["dirty"]: - rendered += "-dirty" - return rendered - - -def render(pieces, style): - """Render the given version pieces into the requested style.""" - if pieces["error"]: - return { - "version": "unknown", - "full-revisionid": pieces.get("long"), - "dirty": None, - "error": pieces["error"], - "date": None, - } - - if not style or style == "default": - style = "pep440" # the default - - if style == "pep440": - rendered = render_pep440(pieces) - elif style == "pep440-pre": - rendered = render_pep440_pre(pieces) - elif style == "pep440-post": - rendered = render_pep440_post(pieces) - elif style == "pep440-old": - rendered = render_pep440_old(pieces) - elif style == "git-describe": - rendered = render_git_describe(pieces) - elif style == "git-describe-long": - rendered = render_git_describe_long(pieces) - else: - raise ValueError("unknown style '%s'" % style) - - return { - "version": rendered, - "full-revisionid": pieces["long"], - "dirty": pieces["dirty"], - "error": None, - "date": pieces.get("date"), - } - - -def get_versions(): - """Get version information or return default if unable to do so.""" - # I am in _version.py, which lives at ROOT/VERSIONFILE_SOURCE. If we have - # __file__, we can work backwards from there to the root. Some - # py2exe/bbfreeze/non-CPython implementations don't do __file__, in which - # case we can only use expanded keywords. - - cfg = get_config() - verbose = cfg.verbose - - try: - return git_versions_from_keywords( - get_keywords(), cfg.tag_prefix, verbose - ) - except NotThisMethod: - pass - - try: - root = os.path.realpath(__file__) - # versionfile_source is the relative path from the top of the source - # tree (where the .git directory might live) to this file. Invert - # this to find the root from __file__. - for i in cfg.versionfile_source.split("/"): - root = os.path.dirname(root) - except NameError: - return { - "version": "0+unknown", - "full-revisionid": None, - "dirty": None, - "error": "unable to find root of source tree", - "date": None, - } - - try: - pieces = git_pieces_from_vcs(cfg.tag_prefix, root, verbose) - return render(pieces, cfg.style) - except NotThisMethod: - pass - - try: - if cfg.parentdir_prefix: - return versions_from_parentdir(cfg.parentdir_prefix, root, verbose) - except NotThisMethod: - pass - - return { - "version": "0+unknown", - "full-revisionid": None, - "dirty": None, - "error": "unable to compute version", - "date": None, - } diff --git a/python/custreamz/setup.cfg b/python/custreamz/setup.cfg index 2ce4eaa82f0..8c038db9349 100644 --- a/python/custreamz/setup.cfg +++ b/python/custreamz/setup.cfg @@ -1,12 +1,4 @@ -# Copyright (c) 2020-2022, NVIDIA CORPORATION. - -[versioneer] -VCS = git -style = pep440 -versionfile_source = custreamz/_version.py -versionfile_build = custreamz/_version.py -tag_prefix = v -parentdir_prefix = custreamz- +# Copyright (c) 2020-2023, NVIDIA CORPORATION. [isort] line_length=79 diff --git a/python/custreamz/setup.py b/python/custreamz/setup.py index 2fe12a54855..65a7aac6395 100644 --- a/python/custreamz/setup.py +++ b/python/custreamz/setup.py @@ -1,6 +1,5 @@ # Copyright (c) 2020-2023, NVIDIA CORPORATION. -import versioneer from setuptools import find_packages, setup install_requires = ["cudf_kafka", "cudf"] @@ -9,7 +8,7 @@ setup( name="custreamz", - version=versioneer.get_version(), + version="23.04.00", description="cuStreamz - GPU Accelerated Streaming", url="https://github.com/rapidsai/cudf", author="NVIDIA Corporation", @@ -26,7 +25,6 @@ "Programming Language :: Python :: 3.10", ], packages=find_packages(include=["custreamz", "custreamz.*"]), - cmdclass=versioneer.get_cmdclass(), install_requires=install_requires, extras_require=extras_require, zip_safe=False, diff --git a/python/custreamz/versioneer.py b/python/custreamz/versioneer.py deleted file mode 100644 index 9c9ddae7340..00000000000 --- a/python/custreamz/versioneer.py +++ /dev/null @@ -1,1904 +0,0 @@ -# Version: 0.18 - -"""The Versioneer - like a rocketeer, but for versions. - -The Versioneer -============== - -* like a rocketeer, but for versions! -* https://github.com/warner/python-versioneer -* Brian Warner -* License: Public Domain -* Compatible With: python2.6, 2.7, 3.2, 3.3, 3.4, 3.5, 3.6, and pypy -* [![Latest Version] -(https://pypip.in/version/versioneer/badge.svg?style=flat) -](https://pypi.python.org/pypi/versioneer/) -* [![Build Status] -(https://travis-ci.org/warner/python-versioneer.png?branch=master) -](https://travis-ci.org/warner/python-versioneer) - -This is a tool for managing a recorded version number in distutils-based -python projects. The goal is to remove the tedious and error-prone "update -the embedded version string" step from your release process. Making a new -release should be as easy as recording a new tag in your version-control -system, and maybe making new tarballs. - - -## Quick Install - -* `pip install versioneer` to somewhere to your $PATH -* add a `[versioneer]` section to your setup.cfg (see below) -* run `versioneer install` in your source tree, commit the results - -## Version Identifiers - -Source trees come from a variety of places: - -* a version-control system checkout (mostly used by developers) -* a nightly tarball, produced by build automation -* a snapshot tarball, produced by a web-based VCS browser, like github's - "tarball from tag" feature -* a release tarball, produced by "setup.py sdist", distributed through PyPI - -Within each source tree, the version identifier (either a string or a number, -this tool is format-agnostic) can come from a variety of places: - -* ask the VCS tool itself, e.g. "git describe" (for checkouts), which knows - about recent "tags" and an absolute revision-id -* the name of the directory into which the tarball was unpacked -* an expanded VCS keyword ($Id$, etc) -* a `_version.py` created by some earlier build step - -For released software, the version identifier is closely related to a VCS -tag. Some projects use tag names that include more than just the version -string (e.g. "myproject-1.2" instead of just "1.2"), in which case the tool -needs to strip the tag prefix to extract the version identifier. For -unreleased software (between tags), the version identifier should provide -enough information to help developers recreate the same tree, while also -giving them an idea of roughly how old the tree is (after version 1.2, before -version 1.3). Many VCS systems can report a description that captures this, -for example `git describe --tags --dirty --always` reports things like -"0.7-1-g574ab98-dirty" to indicate that the checkout is one revision past the -0.7 tag, has a unique revision id of "574ab98", and is "dirty" (it has -uncommitted changes. - -The version identifier is used for multiple purposes: - -* to allow the module to self-identify its version: `myproject.__version__` -* to choose a name and prefix for a 'setup.py sdist' tarball - -## Theory of Operation - -Versioneer works by adding a special `_version.py` file into your source -tree, where your `__init__.py` can import it. This `_version.py` knows how to -dynamically ask the VCS tool for version information at import time. - -`_version.py` also contains `$Revision$` markers, and the installation -process marks `_version.py` to have this marker rewritten with a tag name -during the `git archive` command. As a result, generated tarballs will -contain enough information to get the proper version. - -To allow `setup.py` to compute a version too, a `versioneer.py` is added to -the top level of your source tree, next to `setup.py` and the `setup.cfg` -that configures it. This overrides several distutils/setuptools commands to -compute the version when invoked, and changes `setup.py build` and `setup.py -sdist` to replace `_version.py` with a small static file that contains just -the generated version data. - -## Installation - -See [INSTALL.md](./INSTALL.md) for detailed installation instructions. - -## Version-String Flavors - -Code which uses Versioneer can learn about its version string at runtime by -importing `_version` from your main `__init__.py` file and running the -`get_versions()` function. From the "outside" (e.g. in `setup.py`), you can -import the top-level `versioneer.py` and run `get_versions()`. - -Both functions return a dictionary with different flavors of version -information: - -* `['version']`: A condensed version string, rendered using the selected - style. This is the most commonly used value for the project's version - string. The default "pep440" style yields strings like `0.11`, - `0.11+2.g1076c97`, or `0.11+2.g1076c97.dirty`. See the "Styles" section - below for alternative styles. - -* `['full-revisionid']`: detailed revision identifier. For Git, this is the - full SHA1 commit id, e.g. "1076c978a8d3cfc70f408fe5974aa6c092c949ac". - -* `['date']`: Date and time of the latest `HEAD` commit. For Git, it is the - commit date in ISO 8601 format. This will be None if the date is not - available. - -* `['dirty']`: a boolean, True if the tree has uncommitted changes. Note that - this is only accurate if run in a VCS checkout, otherwise it is likely to - be False or None - -* `['error']`: if the version string could not be computed, this will be set - to a string describing the problem, otherwise it will be None. It may be - useful to throw an exception in setup.py if this is set, to avoid e.g. - creating tarballs with a version string of "unknown". - -Some variants are more useful than others. Including `full-revisionid` in a -bug report should allow developers to reconstruct the exact code being tested -(or indicate the presence of local changes that should be shared with the -developers). `version` is suitable for display in an "about" box or a CLI -`--version` output: it can be easily compared against release notes and lists -of bugs fixed in various releases. - -The installer adds the following text to your `__init__.py` to place a basic -version in `YOURPROJECT.__version__`: - - from custreamz._version import get_versions - __version__ = get_versions()['version'] - del get_versions - -## Styles - -The setup.cfg `style=` configuration controls how the VCS information is -rendered into a version string. - -The default style, "pep440", produces a PEP440-compliant string, equal to the -un-prefixed tag name for actual releases, and containing an additional "local -version" section with more detail for in-between builds. For Git, this is -TAG[+DISTANCE.gHEX[.dirty]] , using information from `git describe --tags ---dirty --always`. For example "0.11+2.g1076c97.dirty" indicates that the -tree is like the "1076c97" commit but has uncommitted changes (".dirty"), and -that this commit is two revisions ("+2") beyond the "0.11" tag. For released -software (exactly equal to a known tag), the identifier will only contain the -stripped tag, e.g. "0.11". - -Other styles are available. See [details.md](details.md) in the Versioneer -source tree for descriptions. - -## Debugging - -Versioneer tries to avoid fatal errors: if something goes wrong, it will tend -to return a version of "0+unknown". To investigate the problem, run `setup.py -version`, which will run the version-lookup code in a verbose mode, and will -display the full contents of `get_versions()` (including the `error` string, -which may help identify what went wrong). - -## Known Limitations - -Some situations are known to cause problems for Versioneer. This details the -most significant ones. More can be found on Github -[issues page](https://github.com/warner/python-versioneer/issues). - -### Subprojects - -Versioneer has limited support for source trees in which `setup.py` is not in -the root directory (e.g. `setup.py` and `.git/` are *not* siblings). The are -two common reasons why `setup.py` might not be in the root: - -* Source trees which contain multiple subprojects, such as - [Buildbot](https://github.com/buildbot/buildbot), which contains both - "master" and "slave" subprojects, each with their own `setup.py`, - `setup.cfg`, and `tox.ini`. Projects like these produce multiple PyPI - distributions (and upload multiple independently-installable tarballs). -* Source trees whose main purpose is to contain a C library, but which also - provide bindings to Python (and perhaps other langauges) in subdirectories. - -Versioneer will look for `.git` in parent directories, and most operations -should get the right version string. However `pip` and `setuptools` have bugs -and implementation details which frequently cause `pip install .` from a -subproject directory to fail to find a correct version string (so it usually -defaults to `0+unknown`). - -`pip install --editable .` should work correctly. `setup.py install` might -work too. - -Pip-8.1.1 is known to have this problem, but hopefully it will get fixed in -some later version. - -[Bug #38](https://github.com/warner/python-versioneer/issues/38) is tracking -this issue. The discussion in -[PR #61](https://github.com/warner/python-versioneer/pull/61) describes the -issue from the Versioneer side in more detail. -[pip PR#3176](https://github.com/pypa/pip/pull/3176) and -[pip PR#3615](https://github.com/pypa/pip/pull/3615) contain work to improve -pip to let Versioneer work correctly. - -Versioneer-0.16 and earlier only looked for a `.git` directory next to the -`setup.cfg`, so subprojects were completely unsupported with those releases. - -### Editable installs with setuptools <= 18.5 - -`setup.py develop` and `pip install --editable .` allow you to install a -project into a virtualenv once, then continue editing the source code (and -test) without re-installing after every change. - -"Entry-point scripts" (`setup(entry_points={"console_scripts": ..})`) are a -convenient way to specify executable scripts that should be installed along -with the python package. - -These both work as expected when using modern setuptools. When using -setuptools-18.5 or earlier, however, certain operations will cause -`pkg_resources.DistributionNotFound` errors when running the entrypoint -script, which must be resolved by re-installing the package. This happens -when the install happens with one version, then the egg_info data is -regenerated while a different version is checked out. Many setup.py commands -cause egg_info to be rebuilt (including `sdist`, `wheel`, and installing into -a different virtualenv), so this can be surprising. - -[Bug #83](https://github.com/warner/python-versioneer/issues/83) describes -this one, but upgrading to a newer version of setuptools should probably -resolve it. - -### Unicode version strings - -While Versioneer works (and is continually tested) with both Python 2 and -Python 3, it is not entirely consistent with bytes-vs-unicode distinctions. -Newer releases probably generate unicode version strings on py2. It's not -clear that this is wrong, but it may be surprising for applications when then -write these strings to a network connection or include them in bytes-oriented -APIs like cryptographic checksums. - -[Bug #71](https://github.com/warner/python-versioneer/issues/71) investigates -this question. - - -## Updating Versioneer - -To upgrade your project to a new release of Versioneer, do the following: - -* install the new Versioneer (`pip install -U versioneer` or equivalent) -* edit `setup.cfg`, if necessary, to include any new configuration settings - indicated by the release notes. See [UPGRADING](./UPGRADING.md) for details. -* re-run `versioneer install` in your source tree, to replace - `SRC/_version.py` -* commit any changed files - -## Future Directions - -This tool is designed to make it easily extended to other version-control -systems: all VCS-specific components are in separate directories like -src/git/ . The top-level `versioneer.py` script is assembled from these -components by running make-versioneer.py . In the future, make-versioneer.py -will take a VCS name as an argument, and will construct a version of -`versioneer.py` that is specific to the given VCS. It might also take the -configuration arguments that are currently provided manually during -installation by editing setup.py . Alternatively, it might go the other -direction and include code from all supported VCS systems, reducing the -number of intermediate scripts. - - -## License - -To make Versioneer easier to embed, all its code is dedicated to the public -domain. The `_version.py` that it creates is also in the public domain. -Specifically, both are released under the Creative Commons "Public Domain -Dedication" license (CC0-1.0), as described in -https://creativecommons.org/publicdomain/zero/1.0/ . - -""" - -from __future__ import print_function - -import errno -import json -import os -import re -import subprocess -import sys - -try: - import configparser -except ImportError: - import ConfigParser as configparser - - -class VersioneerConfig: - """Container for Versioneer configuration parameters.""" - - -def get_root(): - """Get the project root directory. - - We require that all commands are run from the project root, i.e. the - directory that contains setup.py, setup.cfg, and versioneer.py . - """ - root = os.path.realpath(os.path.abspath(os.getcwd())) - setup_py = os.path.join(root, "setup.py") - versioneer_py = os.path.join(root, "versioneer.py") - if not (os.path.exists(setup_py) or os.path.exists(versioneer_py)): - # allow 'python path/to/setup.py COMMAND' - root = os.path.dirname(os.path.realpath(os.path.abspath(sys.argv[0]))) - setup_py = os.path.join(root, "setup.py") - versioneer_py = os.path.join(root, "versioneer.py") - if not (os.path.exists(setup_py) or os.path.exists(versioneer_py)): - err = ( - "Versioneer was unable to run the project root directory. " - "Versioneer requires setup.py to be executed from " - "its immediate directory (like 'python setup.py COMMAND'), " - "or in a way that lets it use sys.argv[0] to find the root " - "(like 'python path/to/setup.py COMMAND')." - ) - raise VersioneerBadRootError(err) - try: - # Certain runtime workflows (setup.py install/develop in a setuptools - # tree) execute all dependencies in a single python process, so - # "versioneer" may be imported multiple times, and python's shared - # module-import table will cache the first one. So we can't use - # os.path.dirname(__file__), as that will find whichever - # versioneer.py was first imported, even in later projects. - me = os.path.realpath(os.path.abspath(__file__)) - me_dir = os.path.normcase(os.path.splitext(me)[0]) - vsr_dir = os.path.normcase(os.path.splitext(versioneer_py)[0]) - if me_dir != vsr_dir: - print( - "Warning: build in %s is using versioneer.py from %s" - % (os.path.dirname(me), versioneer_py) - ) - except NameError: - pass - return root - - -def get_config_from_root(root): - """Read the project setup.cfg file to determine Versioneer config.""" - # This might raise EnvironmentError (if setup.cfg is missing), or - # configparser.NoSectionError (if it lacks a [versioneer] section), or - # configparser.NoOptionError (if it lacks "VCS="). See the docstring at - # the top of versioneer.py for instructions on writing your setup.cfg . - setup_cfg = os.path.join(root, "setup.cfg") - parser = configparser.SafeConfigParser() - with open(setup_cfg, "r") as f: - parser.readfp(f) - VCS = parser.get("versioneer", "VCS") # mandatory - - def get(parser, name): - if parser.has_option("versioneer", name): - return parser.get("versioneer", name) - return None - - cfg = VersioneerConfig() - cfg.VCS = VCS - cfg.style = get(parser, "style") or "" - cfg.versionfile_source = get(parser, "versionfile_source") - cfg.versionfile_build = get(parser, "versionfile_build") - cfg.tag_prefix = get(parser, "tag_prefix") - if cfg.tag_prefix in ("''", '""'): - cfg.tag_prefix = "" - cfg.parentdir_prefix = get(parser, "parentdir_prefix") - cfg.verbose = get(parser, "verbose") - return cfg - - -class NotThisMethod(Exception): - """Exception raised if a method is not valid for the current scenario.""" - - -# these dictionaries contain VCS-specific tools -LONG_VERSION_PY = {} -HANDLERS = {} - - -def register_vcs_handler(vcs, method): # decorator - """Decorator to mark a method as the handler for a particular VCS.""" - - def decorate(f): - """Store f in HANDLERS[vcs][method].""" - if vcs not in HANDLERS: - HANDLERS[vcs] = {} - HANDLERS[vcs][method] = f - return f - - return decorate - - -def run_command( - commands, args, cwd=None, verbose=False, hide_stderr=False, env=None -): - """Call the given command(s).""" - assert isinstance(commands, list) - p = None - for c in commands: - try: - dispcmd = str([c] + args) - # remember shell=False, so use git.cmd on windows, not just git - p = subprocess.Popen( - [c] + args, - cwd=cwd, - env=env, - stdout=subprocess.PIPE, - stderr=(subprocess.PIPE if hide_stderr else None), - ) - break - except EnvironmentError: - e = sys.exc_info()[1] - if e.errno == errno.ENOENT: - continue - if verbose: - print("unable to run %s" % dispcmd) - print(e) - return None, None - else: - if verbose: - print("unable to find command, tried %s" % (commands,)) - return None, None - stdout = p.communicate()[0].strip() - if sys.version_info[0] >= 3: - stdout = stdout.decode() - if p.returncode != 0: - if verbose: - print("unable to run %s (error)" % dispcmd) - print("stdout was %s" % stdout) - return None, p.returncode - return stdout, p.returncode - - -LONG_VERSION_PY[ - "git" -] = r''' -# This file helps to compute a version number in source trees obtained from -# git-archive tarball (such as those provided by githubs download-from-tag -# feature). Distribution tarballs (built by setup.py sdist) and build -# directories (produced by setup.py build) will contain a much shorter file -# that just contains the computed version number. - -# This file is released into the public domain. Generated by -# versioneer-0.18 (https://github.com/warner/python-versioneer) - -"""Git implementation of _version.py.""" - -import errno -import os -import re -import subprocess -import sys - - -def get_keywords(): - """Get the keywords needed to look up the version information.""" - # these strings will be replaced by git during git-archive. - # setup.py/versioneer.py will grep for the variable names, so they must - # each be defined on a line of their own. _version.py will just call - # get_keywords(). - git_refnames = "%(DOLLAR)sFormat:%%d%(DOLLAR)s" - git_full = "%(DOLLAR)sFormat:%%H%(DOLLAR)s" - git_date = "%(DOLLAR)sFormat:%%ci%(DOLLAR)s" - keywords = {"refnames": git_refnames, "full": git_full, "date": git_date} - return keywords - - -class VersioneerConfig: - """Container for Versioneer configuration parameters.""" - - -def get_config(): - """Create, populate and return the VersioneerConfig() object.""" - # these strings are filled in when 'setup.py versioneer' creates - # _version.py - cfg = VersioneerConfig() - cfg.VCS = "git" - cfg.style = "%(STYLE)s" - cfg.tag_prefix = "%(TAG_PREFIX)s" - cfg.parentdir_prefix = "%(PARENTDIR_PREFIX)s" - cfg.versionfile_source = "%(VERSIONFILE_SOURCE)s" - cfg.verbose = False - return cfg - - -class NotThisMethod(Exception): - """Exception raised if a method is not valid for the current scenario.""" - - -LONG_VERSION_PY = {} -HANDLERS = {} - - -def register_vcs_handler(vcs, method): # decorator - """Decorator to mark a method as the handler for a particular VCS.""" - def decorate(f): - """Store f in HANDLERS[vcs][method].""" - if vcs not in HANDLERS: - HANDLERS[vcs] = {} - HANDLERS[vcs][method] = f - return f - return decorate - - -def run_command(commands, args, cwd=None, verbose=False, hide_stderr=False, - env=None): - """Call the given command(s).""" - assert isinstance(commands, list) - p = None - for c in commands: - try: - dispcmd = str([c] + args) - # remember shell=False, so use git.cmd on windows, not just git - p = subprocess.Popen([c] + args, cwd=cwd, env=env, - stdout=subprocess.PIPE, - stderr=(subprocess.PIPE if hide_stderr - else None)) - break - except EnvironmentError: - e = sys.exc_info()[1] - if e.errno == errno.ENOENT: - continue - if verbose: - print("unable to run %%s" %% dispcmd) - print(e) - return None, None - else: - if verbose: - print("unable to find command, tried %%s" %% (commands,)) - return None, None - stdout = p.communicate()[0].strip() - if sys.version_info[0] >= 3: - stdout = stdout.decode() - if p.returncode != 0: - if verbose: - print("unable to run %%s (error)" %% dispcmd) - print("stdout was %%s" %% stdout) - return None, p.returncode - return stdout, p.returncode - - -def versions_from_parentdir(parentdir_prefix, root, verbose): - """Try to determine the version from the parent directory name. - - Source tarballs conventionally unpack into a directory that includes both - the project name and a version string. We will also support searching up - two directory levels for an appropriately named parent directory - """ - rootdirs = [] - - for i in range(3): - dirname = os.path.basename(root) - if dirname.startswith(parentdir_prefix): - return {"version": dirname[len(parentdir_prefix):], - "full-revisionid": None, - "dirty": False, "error": None, "date": None} - else: - rootdirs.append(root) - root = os.path.dirname(root) # up a level - - if verbose: - print("Tried directories %%s but none started with prefix %%s" %% - (str(rootdirs), parentdir_prefix)) - raise NotThisMethod("rootdir doesn't start with parentdir_prefix") - - -@register_vcs_handler("git", "get_keywords") -def git_get_keywords(versionfile_abs): - """Extract version information from the given file.""" - # the code embedded in _version.py can just fetch the value of these - # keywords. When used from setup.py, we don't want to import _version.py, - # so we do it with a regexp instead. This function is not used from - # _version.py. - keywords = {} - try: - f = open(versionfile_abs, "r") - for line in f.readlines(): - if line.strip().startswith("git_refnames ="): - mo = re.search(r'=\s*"(.*)"', line) - if mo: - keywords["refnames"] = mo.group(1) - if line.strip().startswith("git_full ="): - mo = re.search(r'=\s*"(.*)"', line) - if mo: - keywords["full"] = mo.group(1) - if line.strip().startswith("git_date ="): - mo = re.search(r'=\s*"(.*)"', line) - if mo: - keywords["date"] = mo.group(1) - f.close() - except EnvironmentError: - pass - return keywords - - -@register_vcs_handler("git", "keywords") -def git_versions_from_keywords(keywords, tag_prefix, verbose): - """Get version information from git keywords.""" - if not keywords: - raise NotThisMethod("no keywords at all, weird") - date = keywords.get("date") - if date is not None: - # git-2.2.0 added "%%cI", which expands to an ISO-8601 -compliant - # datestamp. However we prefer "%%ci" (which expands to an "ISO-8601 - # -like" string, which we must then edit to make compliant), because - # it's been around since git-1.5.3, and it's too difficult to - # discover which version we're using, or to work around using an - # older one. - date = date.strip().replace(" ", "T", 1).replace(" ", "", 1) - refnames = keywords["refnames"].strip() - if refnames.startswith("$Format"): - if verbose: - print("keywords are unexpanded, not using") - raise NotThisMethod("unexpanded keywords, not a git-archive tarball") - refs = set([r.strip() for r in refnames.strip("()").split(",")]) - # starting in git-1.8.3, tags are listed as "tag: foo-1.0" instead of - # just "foo-1.0". If we see a "tag: " prefix, prefer those. - TAG = "tag: " - tags = set([r[len(TAG):] for r in refs if r.startswith(TAG)]) - if not tags: - # Either we're using git < 1.8.3, or there really are no tags. We use - # a heuristic: assume all version tags have a digit. The old git %%d - # expansion behaves like git log --decorate=short and strips out the - # refs/heads/ and refs/tags/ prefixes that would let us distinguish - # between branches and tags. By ignoring refnames without digits, we - # filter out many common branch names like "release" and - # "stabilization", as well as "HEAD" and "master". - tags = set([r for r in refs if re.search(r'\d', r)]) - if verbose: - print("discarding '%%s', no digits" %% ",".join(refs - tags)) - if verbose: - print("likely tags: %%s" %% ",".join(sorted(tags))) - for ref in sorted(tags): - # sorting will prefer e.g. "2.0" over "2.0rc1" - if ref.startswith(tag_prefix): - r = ref[len(tag_prefix):] - if verbose: - print("picking %%s" %% r) - return {"version": r, - "full-revisionid": keywords["full"].strip(), - "dirty": False, "error": None, - "date": date} - # no suitable tags, so version is "0+unknown", but full hex is still there - if verbose: - print("no suitable tags, using unknown + full revision id") - return {"version": "0+unknown", - "full-revisionid": keywords["full"].strip(), - "dirty": False, "error": "no suitable tags", "date": None} - - -@register_vcs_handler("git", "pieces_from_vcs") -def git_pieces_from_vcs(tag_prefix, root, verbose, run_command=run_command): - """Get version from 'git describe' in the root of the source tree. - - This only gets called if the git-archive 'subst' keywords were *not* - expanded, and _version.py hasn't already been rewritten with a short - version string, meaning we're inside a checked out source tree. - """ - GITS = ["git"] - if sys.platform == "win32": - GITS = ["git.cmd", "git.exe"] - - out, rc = run_command(GITS, ["rev-parse", "--git-dir"], cwd=root, - hide_stderr=True) - if rc != 0: - if verbose: - print("Directory %%s not under git control" %% root) - raise NotThisMethod("'git rev-parse --git-dir' returned error") - - # if there is a tag matching tag_prefix, this yields TAG-NUM-gHEX[-dirty] - # if there isn't one, this yields HEX[-dirty] (no NUM) - describe_out, rc = run_command(GITS, ["describe", "--tags", "--dirty", - "--always", "--long", - "--match", "%%s*" %% tag_prefix], - cwd=root) - # --long was added in git-1.5.5 - if describe_out is None: - raise NotThisMethod("'git describe' failed") - describe_out = describe_out.strip() - full_out, rc = run_command(GITS, ["rev-parse", "HEAD"], cwd=root) - if full_out is None: - raise NotThisMethod("'git rev-parse' failed") - full_out = full_out.strip() - - pieces = {} - pieces["long"] = full_out - pieces["short"] = full_out[:7] # maybe improved later - pieces["error"] = None - - # parse describe_out. It will be like TAG-NUM-gHEX[-dirty] or HEX[-dirty] - # TAG might have hyphens. - git_describe = describe_out - - # look for -dirty suffix - dirty = git_describe.endswith("-dirty") - pieces["dirty"] = dirty - if dirty: - git_describe = git_describe[:git_describe.rindex("-dirty")] - - # now we have TAG-NUM-gHEX or HEX - - if "-" in git_describe: - # TAG-NUM-gHEX - mo = re.search(r'^(.+)-(\d+)-g([0-9a-f]+)$', git_describe) - if not mo: - # unparseable. Maybe git-describe is misbehaving? - pieces["error"] = ("unable to parse git-describe output: '%%s'" - %% describe_out) - return pieces - - # tag - full_tag = mo.group(1) - if not full_tag.startswith(tag_prefix): - if verbose: - fmt = "tag '%%s' doesn't start with prefix '%%s'" - print(fmt %% (full_tag, tag_prefix)) - pieces["error"] = ("tag '%%s' doesn't start with prefix '%%s'" - %% (full_tag, tag_prefix)) - return pieces - pieces["closest-tag"] = full_tag[len(tag_prefix):] - - # distance: number of commits since tag - pieces["distance"] = int(mo.group(2)) - - # commit: short hex revision ID - pieces["short"] = mo.group(3) - - else: - # HEX: no tags - pieces["closest-tag"] = None - count_out, rc = run_command(GITS, ["rev-list", "HEAD", "--count"], - cwd=root) - pieces["distance"] = int(count_out) # total number of commits - - # commit date: see ISO-8601 comment in git_versions_from_keywords() - date = run_command(GITS, ["show", "-s", "--format=%%ci", "HEAD"], - cwd=root)[0].strip() - pieces["date"] = date.strip().replace(" ", "T", 1).replace(" ", "", 1) - - return pieces - - -def plus_or_dot(pieces): - """Return a + if we don't already have one, else return a .""" - if "+" in pieces.get("closest-tag", ""): - return "." - return "+" - - -def render_pep440(pieces): - """Build up version string, with post-release "local version identifier". - - Our goal: TAG[+DISTANCE.gHEX[.dirty]] . Note that if you - get a tagged build and then dirty it, you'll get TAG+0.gHEX.dirty - - Exceptions: - 1: no tags. git_describe was just HEX. 0+untagged.DISTANCE.gHEX[.dirty] - """ - if pieces["closest-tag"]: - rendered = pieces["closest-tag"] - if pieces["distance"] or pieces["dirty"]: - rendered += plus_or_dot(pieces) - rendered += "%%d.g%%s" %% (pieces["distance"], pieces["short"]) - if pieces["dirty"]: - rendered += ".dirty" - else: - # exception #1 - rendered = "0+untagged.%%d.g%%s" %% (pieces["distance"], - pieces["short"]) - if pieces["dirty"]: - rendered += ".dirty" - return rendered - - -def render_pep440_pre(pieces): - """TAG[.post.devDISTANCE] -- No -dirty. - - Exceptions: - 1: no tags. 0.post.devDISTANCE - """ - if pieces["closest-tag"]: - rendered = pieces["closest-tag"] - if pieces["distance"]: - rendered += ".post.dev%%d" %% pieces["distance"] - else: - # exception #1 - rendered = "0.post.dev%%d" %% pieces["distance"] - return rendered - - -def render_pep440_post(pieces): - """TAG[.postDISTANCE[.dev0]+gHEX] . - - The ".dev0" means dirty. Note that .dev0 sorts backwards - (a dirty tree will appear "older" than the corresponding clean one), - but you shouldn't be releasing software with -dirty anyways. - - Exceptions: - 1: no tags. 0.postDISTANCE[.dev0] - """ - if pieces["closest-tag"]: - rendered = pieces["closest-tag"] - if pieces["distance"] or pieces["dirty"]: - rendered += ".post%%d" %% pieces["distance"] - if pieces["dirty"]: - rendered += ".dev0" - rendered += plus_or_dot(pieces) - rendered += "g%%s" %% pieces["short"] - else: - # exception #1 - rendered = "0.post%%d" %% pieces["distance"] - if pieces["dirty"]: - rendered += ".dev0" - rendered += "+g%%s" %% pieces["short"] - return rendered - - -def render_pep440_old(pieces): - """TAG[.postDISTANCE[.dev0]] . - - The ".dev0" means dirty. - - Exceptions: - 1: no tags. 0.postDISTANCE[.dev0] - """ - if pieces["closest-tag"]: - rendered = pieces["closest-tag"] - if pieces["distance"] or pieces["dirty"]: - rendered += ".post%%d" %% pieces["distance"] - if pieces["dirty"]: - rendered += ".dev0" - else: - # exception #1 - rendered = "0.post%%d" %% pieces["distance"] - if pieces["dirty"]: - rendered += ".dev0" - return rendered - - -def render_git_describe(pieces): - """TAG[-DISTANCE-gHEX][-dirty]. - - Like 'git describe --tags --dirty --always'. - - Exceptions: - 1: no tags. HEX[-dirty] (note: no 'g' prefix) - """ - if pieces["closest-tag"]: - rendered = pieces["closest-tag"] - if pieces["distance"]: - rendered += "-%%d-g%%s" %% (pieces["distance"], pieces["short"]) - else: - # exception #1 - rendered = pieces["short"] - if pieces["dirty"]: - rendered += "-dirty" - return rendered - - -def render_git_describe_long(pieces): - """TAG-DISTANCE-gHEX[-dirty]. - - Like 'git describe --tags --dirty --always -long'. - The distance/hash is unconditional. - - Exceptions: - 1: no tags. HEX[-dirty] (note: no 'g' prefix) - """ - if pieces["closest-tag"]: - rendered = pieces["closest-tag"] - rendered += "-%%d-g%%s" %% (pieces["distance"], pieces["short"]) - else: - # exception #1 - rendered = pieces["short"] - if pieces["dirty"]: - rendered += "-dirty" - return rendered - - -def render(pieces, style): - """Render the given version pieces into the requested style.""" - if pieces["error"]: - return {"version": "unknown", - "full-revisionid": pieces.get("long"), - "dirty": None, - "error": pieces["error"], - "date": None} - - if not style or style == "default": - style = "pep440" # the default - - if style == "pep440": - rendered = render_pep440(pieces) - elif style == "pep440-pre": - rendered = render_pep440_pre(pieces) - elif style == "pep440-post": - rendered = render_pep440_post(pieces) - elif style == "pep440-old": - rendered = render_pep440_old(pieces) - elif style == "git-describe": - rendered = render_git_describe(pieces) - elif style == "git-describe-long": - rendered = render_git_describe_long(pieces) - else: - raise ValueError("unknown style '%%s'" %% style) - - return {"version": rendered, "full-revisionid": pieces["long"], - "dirty": pieces["dirty"], "error": None, - "date": pieces.get("date")} - - -def get_versions(): - """Get version information or return default if unable to do so.""" - # I am in _version.py, which lives at ROOT/VERSIONFILE_SOURCE. If we have - # __file__, we can work backwards from there to the root. Some - # py2exe/bbfreeze/non-CPython implementations don't do __file__, in which - # case we can only use expanded keywords. - - cfg = get_config() - verbose = cfg.verbose - - try: - return git_versions_from_keywords(get_keywords(), cfg.tag_prefix, - verbose) - except NotThisMethod: - pass - - try: - root = os.path.realpath(__file__) - # versionfile_source is the relative path from the top of the source - # tree (where the .git directory might live) to this file. Invert - # this to find the root from __file__. - for i in cfg.versionfile_source.split('/'): - root = os.path.dirname(root) - except NameError: - return {"version": "0+unknown", "full-revisionid": None, - "dirty": None, - "error": "unable to find root of source tree", - "date": None} - - try: - pieces = git_pieces_from_vcs(cfg.tag_prefix, root, verbose) - return render(pieces, cfg.style) - except NotThisMethod: - pass - - try: - if cfg.parentdir_prefix: - return versions_from_parentdir(cfg.parentdir_prefix, root, verbose) - except NotThisMethod: - pass - - return {"version": "0+unknown", "full-revisionid": None, - "dirty": None, - "error": "unable to compute version", "date": None} -''' - - -@register_vcs_handler("git", "get_keywords") -def git_get_keywords(versionfile_abs): - """Extract version information from the given file.""" - # the code embedded in _version.py can just fetch the value of these - # keywords. When used from setup.py, we don't want to import _version.py, - # so we do it with a regexp instead. This function is not used from - # _version.py. - keywords = {} - try: - f = open(versionfile_abs, "r") - for line in f.readlines(): - if line.strip().startswith("git_refnames ="): - mo = re.search(r'=\s*"(.*)"', line) - if mo: - keywords["refnames"] = mo.group(1) - if line.strip().startswith("git_full ="): - mo = re.search(r'=\s*"(.*)"', line) - if mo: - keywords["full"] = mo.group(1) - if line.strip().startswith("git_date ="): - mo = re.search(r'=\s*"(.*)"', line) - if mo: - keywords["date"] = mo.group(1) - f.close() - except EnvironmentError: - pass - return keywords - - -@register_vcs_handler("git", "keywords") -def git_versions_from_keywords(keywords, tag_prefix, verbose): - """Get version information from git keywords.""" - if not keywords: - raise NotThisMethod("no keywords at all, weird") - date = keywords.get("date") - if date is not None: - # git-2.2.0 added "%cI", which expands to an ISO-8601 -compliant - # datestamp. However we prefer "%ci" (which expands to an "ISO-8601 - # -like" string, which we must then edit to make compliant), because - # it's been around since git-1.5.3, and it's too difficult to - # discover which version we're using, or to work around using an - # older one. - date = date.strip().replace(" ", "T", 1).replace(" ", "", 1) - refnames = keywords["refnames"].strip() - if refnames.startswith("$Format"): - if verbose: - print("keywords are unexpanded, not using") - raise NotThisMethod("unexpanded keywords, not a git-archive tarball") - refs = set([r.strip() for r in refnames.strip("()").split(",")]) - # starting in git-1.8.3, tags are listed as "tag: foo-1.0" instead of - # just "foo-1.0". If we see a "tag: " prefix, prefer those. - TAG = "tag: " - tags = set([r[len(TAG) :] for r in refs if r.startswith(TAG)]) - if not tags: - # Either we're using git < 1.8.3, or there really are no tags. We use - # a heuristic: assume all version tags have a digit. The old git %d - # expansion behaves like git log --decorate=short and strips out the - # refs/heads/ and refs/tags/ prefixes that would let us distinguish - # between branches and tags. By ignoring refnames without digits, we - # filter out many common branch names like "release" and - # "stabilization", as well as "HEAD" and "master". - tags = set([r for r in refs if re.search(r"\d", r)]) - if verbose: - print("discarding '%s', no digits" % ",".join(refs - tags)) - if verbose: - print("likely tags: %s" % ",".join(sorted(tags))) - for ref in sorted(tags): - # sorting will prefer e.g. "2.0" over "2.0rc1" - if ref.startswith(tag_prefix): - r = ref[len(tag_prefix) :] - if verbose: - print("picking %s" % r) - return { - "version": r, - "full-revisionid": keywords["full"].strip(), - "dirty": False, - "error": None, - "date": date, - } - # no suitable tags, so version is "0+unknown", but full hex is still there - if verbose: - print("no suitable tags, using unknown + full revision id") - return { - "version": "0+unknown", - "full-revisionid": keywords["full"].strip(), - "dirty": False, - "error": "no suitable tags", - "date": None, - } - - -@register_vcs_handler("git", "pieces_from_vcs") -def git_pieces_from_vcs(tag_prefix, root, verbose, run_command=run_command): - """Get version from 'git describe' in the root of the source tree. - - This only gets called if the git-archive 'subst' keywords were *not* - expanded, and _version.py hasn't already been rewritten with a short - version string, meaning we're inside a checked out source tree. - """ - GITS = ["git"] - if sys.platform == "win32": - GITS = ["git.cmd", "git.exe"] - - out, rc = run_command( - GITS, ["rev-parse", "--git-dir"], cwd=root, hide_stderr=True - ) - if rc != 0: - if verbose: - print("Directory %s not under git control" % root) - raise NotThisMethod("'git rev-parse --git-dir' returned error") - - # if there is a tag matching tag_prefix, this yields TAG-NUM-gHEX[-dirty] - # if there isn't one, this yields HEX[-dirty] (no NUM) - describe_out, rc = run_command( - GITS, - [ - "describe", - "--tags", - "--dirty", - "--always", - "--long", - "--match", - "%s*" % tag_prefix, - ], - cwd=root, - ) - # --long was added in git-1.5.5 - if describe_out is None: - raise NotThisMethod("'git describe' failed") - describe_out = describe_out.strip() - full_out, rc = run_command(GITS, ["rev-parse", "HEAD"], cwd=root) - if full_out is None: - raise NotThisMethod("'git rev-parse' failed") - full_out = full_out.strip() - - pieces = {} - pieces["long"] = full_out - pieces["short"] = full_out[:7] # maybe improved later - pieces["error"] = None - - # parse describe_out. It will be like TAG-NUM-gHEX[-dirty] or HEX[-dirty] - # TAG might have hyphens. - git_describe = describe_out - - # look for -dirty suffix - dirty = git_describe.endswith("-dirty") - pieces["dirty"] = dirty - if dirty: - git_describe = git_describe[: git_describe.rindex("-dirty")] - - # now we have TAG-NUM-gHEX or HEX - - if "-" in git_describe: - # TAG-NUM-gHEX - mo = re.search(r"^(.+)-(\d+)-g([0-9a-f]+)$", git_describe) - if not mo: - # unparseable. Maybe git-describe is misbehaving? - pieces["error"] = ( - "unable to parse git-describe output: '%s'" % describe_out - ) - return pieces - - # tag - full_tag = mo.group(1) - if not full_tag.startswith(tag_prefix): - if verbose: - fmt = "tag '%s' doesn't start with prefix '%s'" - print(fmt % (full_tag, tag_prefix)) - pieces["error"] = "tag '%s' doesn't start with prefix '%s'" % ( - full_tag, - tag_prefix, - ) - return pieces - pieces["closest-tag"] = full_tag[len(tag_prefix) :] - - # distance: number of commits since tag - pieces["distance"] = int(mo.group(2)) - - # commit: short hex revision ID - pieces["short"] = mo.group(3) - - else: - # HEX: no tags - pieces["closest-tag"] = None - count_out, rc = run_command( - GITS, ["rev-list", "HEAD", "--count"], cwd=root - ) - pieces["distance"] = int(count_out) # total number of commits - - # commit date: see ISO-8601 comment in git_versions_from_keywords() - date = run_command(GITS, ["show", "-s", "--format=%ci", "HEAD"], cwd=root)[ - 0 - ].strip() - pieces["date"] = date.strip().replace(" ", "T", 1).replace(" ", "", 1) - - return pieces - - -def do_vcs_install(manifest_in, versionfile_source, ipy): - """Git-specific installation logic for Versioneer. - - For Git, this means creating/changing .gitattributes to mark _version.py - for export-subst keyword substitution. - """ - GITS = ["git"] - if sys.platform == "win32": - GITS = ["git.cmd", "git.exe"] - files = [manifest_in, versionfile_source] - if ipy: - files.append(ipy) - try: - me = __file__ - if me.endswith(".pyc") or me.endswith(".pyo"): - me = os.path.splitext(me)[0] + ".py" - versioneer_file = os.path.relpath(me) - except NameError: - versioneer_file = "versioneer.py" - files.append(versioneer_file) - present = False - try: - f = open(".gitattributes", "r") - for line in f.readlines(): - if line.strip().startswith(versionfile_source): - if "export-subst" in line.strip().split()[1:]: - present = True - f.close() - except EnvironmentError: - pass - if not present: - f = open(".gitattributes", "a+") - f.write("%s export-subst\n" % versionfile_source) - f.close() - files.append(".gitattributes") - run_command(GITS, ["add", "--"] + files) - - -def versions_from_parentdir(parentdir_prefix, root, verbose): - """Try to determine the version from the parent directory name. - - Source tarballs conventionally unpack into a directory that includes both - the project name and a version string. We will also support searching up - two directory levels for an appropriately named parent directory - """ - rootdirs = [] - - for i in range(3): - dirname = os.path.basename(root) - if dirname.startswith(parentdir_prefix): - return { - "version": dirname[len(parentdir_prefix) :], - "full-revisionid": None, - "dirty": False, - "error": None, - "date": None, - } - else: - rootdirs.append(root) - root = os.path.dirname(root) # up a level - - if verbose: - print( - "Tried directories %s but none started with prefix %s" - % (str(rootdirs), parentdir_prefix) - ) - raise NotThisMethod("rootdir doesn't start with parentdir_prefix") - - -SHORT_VERSION_PY = """ -# This file was generated by 'versioneer.py' (0.18) from -# revision-control system data, or from the parent directory name of an -# unpacked source archive. Distribution tarballs contain a pre-generated copy -# of this file. - -import json - -version_json = ''' -%s -''' # END VERSION_JSON - - -def get_versions(): - return json.loads(version_json) -""" - - -def versions_from_file(filename): - """Try to determine the version from _version.py if present.""" - try: - with open(filename) as f: - contents = f.read() - except EnvironmentError: - raise NotThisMethod("unable to read _version.py") - mo = re.search( - r"version_json = '''\n(.*)''' # END VERSION_JSON", - contents, - re.M | re.S, - ) - if not mo: - mo = re.search( - r"version_json = '''\r\n(.*)''' # END VERSION_JSON", - contents, - re.M | re.S, - ) - if not mo: - raise NotThisMethod("no version_json in _version.py") - return json.loads(mo.group(1)) - - -def write_to_version_file(filename, versions): - """Write the given version number to the given _version.py file.""" - os.unlink(filename) - contents = json.dumps( - versions, sort_keys=True, indent=1, separators=(",", ": ") - ) - with open(filename, "w") as f: - f.write(SHORT_VERSION_PY % contents) - - print("set %s to '%s'" % (filename, versions["version"])) - - -def plus_or_dot(pieces): - """Return a + if we don't already have one, else return a .""" - if "+" in pieces.get("closest-tag", ""): - return "." - return "+" - - -def render_pep440(pieces): - """Build up version string, with post-release "local version identifier". - - Our goal: TAG[+DISTANCE.gHEX[.dirty]] . Note that if you - get a tagged build and then dirty it, you'll get TAG+0.gHEX.dirty - - Exceptions: - 1: no tags. git_describe was just HEX. 0+untagged.DISTANCE.gHEX[.dirty] - """ - if pieces["closest-tag"]: - rendered = pieces["closest-tag"] - if pieces["distance"] or pieces["dirty"]: - rendered += plus_or_dot(pieces) - rendered += "%d.g%s" % (pieces["distance"], pieces["short"]) - if pieces["dirty"]: - rendered += ".dirty" - else: - # exception #1 - rendered = "0+untagged.%d.g%s" % (pieces["distance"], pieces["short"]) - if pieces["dirty"]: - rendered += ".dirty" - return rendered - - -def render_pep440_pre(pieces): - """TAG[.post.devDISTANCE] -- No -dirty. - - Exceptions: - 1: no tags. 0.post.devDISTANCE - """ - if pieces["closest-tag"]: - rendered = pieces["closest-tag"] - if pieces["distance"]: - rendered += ".post.dev%d" % pieces["distance"] - else: - # exception #1 - rendered = "0.post.dev%d" % pieces["distance"] - return rendered - - -def render_pep440_post(pieces): - """TAG[.postDISTANCE[.dev0]+gHEX] . - - The ".dev0" means dirty. Note that .dev0 sorts backwards - (a dirty tree will appear "older" than the corresponding clean one), - but you shouldn't be releasing software with -dirty anyways. - - Exceptions: - 1: no tags. 0.postDISTANCE[.dev0] - """ - if pieces["closest-tag"]: - rendered = pieces["closest-tag"] - if pieces["distance"] or pieces["dirty"]: - rendered += ".post%d" % pieces["distance"] - if pieces["dirty"]: - rendered += ".dev0" - rendered += plus_or_dot(pieces) - rendered += "g%s" % pieces["short"] - else: - # exception #1 - rendered = "0.post%d" % pieces["distance"] - if pieces["dirty"]: - rendered += ".dev0" - rendered += "+g%s" % pieces["short"] - return rendered - - -def render_pep440_old(pieces): - """TAG[.postDISTANCE[.dev0]] . - - The ".dev0" means dirty. - - Exceptions: - 1: no tags. 0.postDISTANCE[.dev0] - """ - if pieces["closest-tag"]: - rendered = pieces["closest-tag"] - if pieces["distance"] or pieces["dirty"]: - rendered += ".post%d" % pieces["distance"] - if pieces["dirty"]: - rendered += ".dev0" - else: - # exception #1 - rendered = "0.post%d" % pieces["distance"] - if pieces["dirty"]: - rendered += ".dev0" - return rendered - - -def render_git_describe(pieces): - """TAG[-DISTANCE-gHEX][-dirty]. - - Like 'git describe --tags --dirty --always'. - - Exceptions: - 1: no tags. HEX[-dirty] (note: no 'g' prefix) - """ - if pieces["closest-tag"]: - rendered = pieces["closest-tag"] - if pieces["distance"]: - rendered += "-%d-g%s" % (pieces["distance"], pieces["short"]) - else: - # exception #1 - rendered = pieces["short"] - if pieces["dirty"]: - rendered += "-dirty" - return rendered - - -def render_git_describe_long(pieces): - """TAG-DISTANCE-gHEX[-dirty]. - - Like 'git describe --tags --dirty --always -long'. - The distance/hash is unconditional. - - Exceptions: - 1: no tags. HEX[-dirty] (note: no 'g' prefix) - """ - if pieces["closest-tag"]: - rendered = pieces["closest-tag"] - rendered += "-%d-g%s" % (pieces["distance"], pieces["short"]) - else: - # exception #1 - rendered = pieces["short"] - if pieces["dirty"]: - rendered += "-dirty" - return rendered - - -def render(pieces, style): - """Render the given version pieces into the requested style.""" - if pieces["error"]: - return { - "version": "unknown", - "full-revisionid": pieces.get("long"), - "dirty": None, - "error": pieces["error"], - "date": None, - } - - if not style or style == "default": - style = "pep440" # the default - - if style == "pep440": - rendered = render_pep440(pieces) - elif style == "pep440-pre": - rendered = render_pep440_pre(pieces) - elif style == "pep440-post": - rendered = render_pep440_post(pieces) - elif style == "pep440-old": - rendered = render_pep440_old(pieces) - elif style == "git-describe": - rendered = render_git_describe(pieces) - elif style == "git-describe-long": - rendered = render_git_describe_long(pieces) - else: - raise ValueError("unknown style '%s'" % style) - - return { - "version": rendered, - "full-revisionid": pieces["long"], - "dirty": pieces["dirty"], - "error": None, - "date": pieces.get("date"), - } - - -class VersioneerBadRootError(Exception): - """The project root directory is unknown or missing key files.""" - - -def get_versions(verbose=False): - """Get the project version from whatever source is available. - - Returns dict with two keys: 'version' and 'full'. - """ - if "versioneer" in sys.modules: - # see the discussion in cmdclass.py:get_cmdclass() - del sys.modules["versioneer"] - - root = get_root() - cfg = get_config_from_root(root) - - assert cfg.VCS is not None, "please set [versioneer]VCS= in setup.cfg" - handlers = HANDLERS.get(cfg.VCS) - assert handlers, "unrecognized VCS '%s'" % cfg.VCS - verbose = verbose or cfg.verbose - assert ( - cfg.versionfile_source is not None - ), "please set versioneer.versionfile_source" - assert cfg.tag_prefix is not None, "please set versioneer.tag_prefix" - - versionfile_abs = os.path.join(root, cfg.versionfile_source) - - # extract version from first of: _version.py, VCS command (e.g. 'git - # describe'), parentdir. This is meant to work for developers using a - # source checkout, for users of a tarball created by 'setup.py sdist', - # and for users of a tarball/zipball created by 'git archive' or github's - # download-from-tag feature or the equivalent in other VCSes. - - get_keywords_f = handlers.get("get_keywords") - from_keywords_f = handlers.get("keywords") - if get_keywords_f and from_keywords_f: - try: - keywords = get_keywords_f(versionfile_abs) - ver = from_keywords_f(keywords, cfg.tag_prefix, verbose) - if verbose: - print("got version from expanded keyword %s" % ver) - return ver - except NotThisMethod: - pass - - try: - ver = versions_from_file(versionfile_abs) - if verbose: - print("got version from file %s %s" % (versionfile_abs, ver)) - return ver - except NotThisMethod: - pass - - from_vcs_f = handlers.get("pieces_from_vcs") - if from_vcs_f: - try: - pieces = from_vcs_f(cfg.tag_prefix, root, verbose) - ver = render(pieces, cfg.style) - if verbose: - print("got version from VCS %s" % ver) - return ver - except NotThisMethod: - pass - - try: - if cfg.parentdir_prefix: - ver = versions_from_parentdir(cfg.parentdir_prefix, root, verbose) - if verbose: - print("got version from parentdir %s" % ver) - return ver - except NotThisMethod: - pass - - if verbose: - print("unable to compute version") - - return { - "version": "0+unknown", - "full-revisionid": None, - "dirty": None, - "error": "unable to compute version", - "date": None, - } - - -def get_version(): - """Get the short version string for this project.""" - return get_versions()["version"] - - -def get_cmdclass(): - """Get the custom setuptools/distutils subclasses used by Versioneer.""" - if "versioneer" in sys.modules: - del sys.modules["versioneer"] - # this fixes the "python setup.py develop" case (also 'install' and - # 'easy_install .'), in which subdependencies of the main project are - # built (using setup.py bdist_egg) in the same python process. Assume - # a main project A and a dependency B, which use different versions - # of Versioneer. A's setup.py imports A's Versioneer, leaving it in - # sys.modules by the time B's setup.py is executed, causing B to run - # with the wrong versioneer. Setuptools wraps the sub-dep builds in a - # sandbox that restores sys.modules to it's pre-build state, so the - # parent is protected against the child's "import versioneer". By - # removing ourselves from sys.modules here, before the child build - # happens, we protect the child from the parent's versioneer too. - # Also see https://github.com/warner/python-versioneer/issues/52 - - cmds = {} - - # we add "version" to both distutils and setuptools - from distutils.core import Command - - class cmd_version(Command): - description = "report generated version string" - user_options = [] - boolean_options = [] - - def initialize_options(self): - pass - - def finalize_options(self): - pass - - def run(self): - vers = get_versions(verbose=True) - print("Version: %s" % vers["version"]) - print(" full-revisionid: %s" % vers.get("full-revisionid")) - print(" dirty: %s" % vers.get("dirty")) - print(" date: %s" % vers.get("date")) - if vers["error"]: - print(" error: %s" % vers["error"]) - - cmds["version"] = cmd_version - - # we override "build_py" in both distutils and setuptools - # - # most invocation pathways end up running build_py: - # distutils/build -> build_py - # distutils/install -> distutils/build ->.. - # setuptools/bdist_wheel -> distutils/install ->.. - # setuptools/bdist_egg -> distutils/install_lib -> build_py - # setuptools/install -> bdist_egg ->.. - # setuptools/develop -> ? - # pip install: - # copies source tree to a tempdir before running egg_info/etc - # if .git isn't copied too, 'git describe' will fail - # then does setup.py bdist_wheel, or sometimes setup.py install - # setup.py egg_info -> ? - - # we override different "build_py" commands for both environments - if "setuptools" in sys.modules: - from setuptools.command.build_py import build_py as _build_py - else: - from distutils.command.build_py import build_py as _build_py - - class cmd_build_py(_build_py): - def run(self): - root = get_root() - cfg = get_config_from_root(root) - versions = get_versions() - _build_py.run(self) - # now locate _version.py in the new build/ directory and replace - # it with an updated value - if cfg.versionfile_build: - target_versionfile = os.path.join( - self.build_lib, cfg.versionfile_build - ) - print("UPDATING %s" % target_versionfile) - write_to_version_file(target_versionfile, versions) - - cmds["build_py"] = cmd_build_py - - if "cx_Freeze" in sys.modules: # cx_freeze enabled? - from cx_Freeze.dist import build_exe as _build_exe - - # nczeczulin reports that py2exe won't like the pep440-style string - # as FILEVERSION, but it can be used for PRODUCTVERSION, e.g. - # setup(console=[{ - # "version": versioneer.get_version().split("+", 1)[0], # FILEVERSION - # "product_version": versioneer.get_version(), - # ... - - class cmd_build_exe(_build_exe): - def run(self): - root = get_root() - cfg = get_config_from_root(root) - versions = get_versions() - target_versionfile = cfg.versionfile_source - print("UPDATING %s" % target_versionfile) - write_to_version_file(target_versionfile, versions) - - _build_exe.run(self) - os.unlink(target_versionfile) - with open(cfg.versionfile_source, "w") as f: - LONG = LONG_VERSION_PY[cfg.VCS] - f.write( - LONG - % { - "DOLLAR": "$", - "STYLE": cfg.style, - "TAG_PREFIX": cfg.tag_prefix, - "PARENTDIR_PREFIX": cfg.parentdir_prefix, - "VERSIONFILE_SOURCE": cfg.versionfile_source, - } - ) - - cmds["build_exe"] = cmd_build_exe - del cmds["build_py"] - - if "py2exe" in sys.modules: # py2exe enabled? - try: - from py2exe.distutils_buildexe import py2exe as _py2exe # py3 - except ImportError: - from py2exe.build_exe import py2exe as _py2exe # py2 - - class cmd_py2exe(_py2exe): - def run(self): - root = get_root() - cfg = get_config_from_root(root) - versions = get_versions() - target_versionfile = cfg.versionfile_source - print("UPDATING %s" % target_versionfile) - write_to_version_file(target_versionfile, versions) - - _py2exe.run(self) - os.unlink(target_versionfile) - with open(cfg.versionfile_source, "w") as f: - LONG = LONG_VERSION_PY[cfg.VCS] - f.write( - LONG - % { - "DOLLAR": "$", - "STYLE": cfg.style, - "TAG_PREFIX": cfg.tag_prefix, - "PARENTDIR_PREFIX": cfg.parentdir_prefix, - "VERSIONFILE_SOURCE": cfg.versionfile_source, - } - ) - - cmds["py2exe"] = cmd_py2exe - - # we override different "sdist" commands for both environments - if "setuptools" in sys.modules: - from setuptools.command.sdist import sdist as _sdist - else: - from distutils.command.sdist import sdist as _sdist - - class cmd_sdist(_sdist): - def run(self): - versions = get_versions() - self._versioneer_generated_versions = versions - # unless we update this, the command will keep using the old - # version - self.distribution.metadata.version = versions["version"] - return _sdist.run(self) - - def make_release_tree(self, base_dir, files): - root = get_root() - cfg = get_config_from_root(root) - _sdist.make_release_tree(self, base_dir, files) - # now locate _version.py in the new base_dir directory - # (remembering that it may be a hardlink) and replace it with an - # updated value - target_versionfile = os.path.join(base_dir, cfg.versionfile_source) - print("UPDATING %s" % target_versionfile) - write_to_version_file( - target_versionfile, self._versioneer_generated_versions - ) - - cmds["sdist"] = cmd_sdist - - return cmds - - -CONFIG_ERROR = """ -setup.cfg is missing the necessary Versioneer configuration. You need -a section like: - - [versioneer] - VCS = git - style = pep440 - versionfile_source = src/myproject/_version.py - versionfile_build = myproject/_version.py - tag_prefix = - parentdir_prefix = myproject- - -You will also need to edit your setup.py to use the results: - - import versioneer - setup(version=versioneer.get_version(), - cmdclass=versioneer.get_cmdclass(), ...) - -Please read the docstring in ./versioneer.py for configuration instructions, -edit setup.cfg, and re-run the installer or 'python versioneer.py setup'. -""" - -SAMPLE_CONFIG = """ -# See the docstring in versioneer.py for instructions. Note that you must -# re-run 'versioneer.py setup' after changing this section, and commit the -# resulting files. - -[versioneer] -#VCS = git -#style = pep440 -#versionfile_source = -#versionfile_build = -#tag_prefix = -#parentdir_prefix = - -""" - -INIT_PY_SNIPPET = """ -from custreamz._version import get_versions -__version__ = get_versions()['version'] -del get_versions -""" - - -def do_setup(): - """Main VCS-independent setup function for installing Versioneer.""" - root = get_root() - try: - cfg = get_config_from_root(root) - except ( - EnvironmentError, - configparser.NoSectionError, - configparser.NoOptionError, - ) as e: - if isinstance(e, (EnvironmentError, configparser.NoSectionError)): - print( - "Adding sample versioneer config to setup.cfg", file=sys.stderr - ) - with open(os.path.join(root, "setup.cfg"), "a") as f: - f.write(SAMPLE_CONFIG) - print(CONFIG_ERROR, file=sys.stderr) - return 1 - - print(" creating %s" % cfg.versionfile_source) - with open(cfg.versionfile_source, "w") as f: - LONG = LONG_VERSION_PY[cfg.VCS] - f.write( - LONG - % { - "DOLLAR": "$", - "STYLE": cfg.style, - "TAG_PREFIX": cfg.tag_prefix, - "PARENTDIR_PREFIX": cfg.parentdir_prefix, - "VERSIONFILE_SOURCE": cfg.versionfile_source, - } - ) - - ipy = os.path.join(os.path.dirname(cfg.versionfile_source), "__init__.py") - if os.path.exists(ipy): - try: - with open(ipy, "r") as f: - old = f.read() - except EnvironmentError: - old = "" - if INIT_PY_SNIPPET not in old: - print(" appending to %s" % ipy) - with open(ipy, "a") as f: - f.write(INIT_PY_SNIPPET) - else: - print(" %s unmodified" % ipy) - else: - print(" %s doesn't exist, ok" % ipy) - ipy = None - - # Make sure both the top-level "versioneer.py" and versionfile_source - # (PKG/_version.py, used by runtime code) are in MANIFEST.in, so - # they'll be copied into source distributions. Pip won't be able to - # install the package without this. - manifest_in = os.path.join(root, "MANIFEST.in") - simple_includes = set() - try: - with open(manifest_in, "r") as f: - for line in f: - if line.startswith("include "): - for include in line.split()[1:]: - simple_includes.add(include) - except EnvironmentError: - pass - # That doesn't cover everything MANIFEST.in can do - # (http://docs.python.org/2/distutils/sourcedist.html#commands), so - # it might give some false negatives. Appending redundant 'include' - # lines is safe, though. - if "versioneer.py" not in simple_includes: - print(" appending 'versioneer.py' to MANIFEST.in") - with open(manifest_in, "a") as f: - f.write("include versioneer.py\n") - else: - print(" 'versioneer.py' already in MANIFEST.in") - if cfg.versionfile_source not in simple_includes: - print( - " appending versionfile_source ('%s') to MANIFEST.in" - % cfg.versionfile_source - ) - with open(manifest_in, "a") as f: - f.write("include %s\n" % cfg.versionfile_source) - else: - print(" versionfile_source already in MANIFEST.in") - - # Make VCS-specific changes. For git, this means creating/changing - # .gitattributes to mark _version.py for export-subst keyword - # substitution. - do_vcs_install(manifest_in, cfg.versionfile_source, ipy) - return 0 - - -def scan_setup_py(): - """Validate the contents of setup.py against Versioneer's expectations.""" - found = set() - setters = False - errors = 0 - with open("setup.py", "r") as f: - for line in f.readlines(): - if "import versioneer" in line: - found.add("import") - if "versioneer.get_cmdclass()" in line: - found.add("cmdclass") - if "versioneer.get_version()" in line: - found.add("get_version") - if "versioneer.VCS" in line: - setters = True - if "versioneer.versionfile_source" in line: - setters = True - if len(found) != 3: - print("") - print("Your setup.py appears to be missing some important items") - print("(but I might be wrong). Please make sure it has something") - print("roughly like the following:") - print("") - print(" import versioneer") - print(" setup( version=versioneer.get_version(),") - print(" cmdclass=versioneer.get_cmdclass(), ...)") - print("") - errors += 1 - if setters: - print("You should remove lines like 'versioneer.VCS = ' and") - print("'versioneer.versionfile_source = ' . This configuration") - print("now lives in setup.cfg, and should be removed from setup.py") - print("") - errors += 1 - return errors - - -if __name__ == "__main__": - cmd = sys.argv[1] - if cmd == "setup": - errors = do_setup() - errors += scan_setup_py() - if errors: - sys.exit(1) diff --git a/python/dask_cudf/dask_cudf/__init__.py b/python/dask_cudf/dask_cudf/__init__.py index 5e3a9342c25..010e4a104b2 100644 --- a/python/dask_cudf/dask_cudf/__init__.py +++ b/python/dask_cudf/dask_cudf/__init__.py @@ -1,9 +1,8 @@ -# Copyright (c) 2018-2022, NVIDIA CORPORATION. +# Copyright (c) 2018-2023, NVIDIA CORPORATION. from dask.dataframe import from_delayed import cudf -from cudf._version import get_versions from . import backends from .core import DataFrame, Series, concat, from_cudf, from_dask_dataframe @@ -15,8 +14,7 @@ except ImportError: pass -__version__ = get_versions()["version"] -del get_versions +__version__ = "23.04.00" __all__ = [ "DataFrame", diff --git a/python/dask_cudf/dask_cudf/_version.py b/python/dask_cudf/dask_cudf/_version.py deleted file mode 100644 index f0dbcac0017..00000000000 --- a/python/dask_cudf/dask_cudf/_version.py +++ /dev/null @@ -1,566 +0,0 @@ -# Copyright (c) 2020-2022, NVIDIA CORPORATION. -# This file helps to compute a version number in source trees obtained from -# git-archive tarball (such as those provided by githubs download-from-tag -# feature). Distribution tarballs (built by setup.py sdist) and build -# directories (produced by setup.py build) will contain a much shorter file -# that just contains the computed version number. - -# This file is released into the public domain. Generated by -# versioneer-0.18 (https://github.com/warner/python-versioneer) - -"""Git implementation of _version.py.""" - -import errno -import os -import re -import subprocess -import sys - - -def get_keywords(): - """Get the keywords needed to look up the version information.""" - # these strings will be replaced by git during git-archive. - # setup.py/versioneer.py will grep for the variable names, so they must - # each be defined on a line of their own. _version.py will just call - # get_keywords(). - git_refnames = "$Format:%d$" - git_full = "$Format:%H$" - git_date = "$Format:%ci$" - keywords = {"refnames": git_refnames, "full": git_full, "date": git_date} - return keywords - - -class VersioneerConfig: - """Container for Versioneer configuration parameters.""" - - -def get_config(): - """Create, populate and return the VersioneerConfig() object.""" - # these strings are filled in when 'setup.py versioneer' creates - # _version.py - cfg = VersioneerConfig() - cfg.VCS = "git" - cfg.style = "pep440" - cfg.tag_prefix = "" - cfg.parentdir_prefix = "dask_cudf-" - cfg.versionfile_source = "dask_cudf/_version.py" - cfg.verbose = False - return cfg - - -class NotThisMethod(Exception): - """Exception raised if a method is not valid for the current scenario.""" - - -LONG_VERSION_PY = {} -HANDLERS = {} - - -def register_vcs_handler(vcs, method): # decorator - """Decorator to mark a method as the handler for a particular VCS.""" - - def decorate(f): - """Store f in HANDLERS[vcs][method].""" - if vcs not in HANDLERS: - HANDLERS[vcs] = {} - HANDLERS[vcs][method] = f - return f - - return decorate - - -def run_command( - commands, args, cwd=None, verbose=False, hide_stderr=False, env=None -): - """Call the given command(s).""" - assert isinstance(commands, list) - p = None - for c in commands: - try: - dispcmd = str([c] + args) - # remember shell=False, so use git.cmd on windows, not just git - p = subprocess.Popen( - [c] + args, - cwd=cwd, - env=env, - stdout=subprocess.PIPE, - stderr=(subprocess.PIPE if hide_stderr else None), - ) - break - except OSError: - e = sys.exc_info()[1] - if e.errno == errno.ENOENT: - continue - if verbose: - print("unable to run %s" % dispcmd) - print(e) - return None, None - else: - if verbose: - print(f"unable to find command, tried {commands}") - return None, None - stdout = p.communicate()[0].strip() - if sys.version_info[0] >= 3: - stdout = stdout.decode() - if p.returncode != 0: - if verbose: - print("unable to run %s (error)" % dispcmd) - print("stdout was %s" % stdout) - return None, p.returncode - return stdout, p.returncode - - -def versions_from_parentdir(parentdir_prefix, root, verbose): - """Try to determine the version from the parent directory name. - - Source tarballs conventionally unpack into a directory that includes both - the project name and a version string. We will also support searching up - two directory levels for an appropriately named parent directory - """ - rootdirs = [] - - for i in range(3): - dirname = os.path.basename(root) - if dirname.startswith(parentdir_prefix): - return { - "version": dirname[len(parentdir_prefix) :], - "full-revisionid": None, - "dirty": False, - "error": None, - "date": None, - } - else: - rootdirs.append(root) - root = os.path.dirname(root) # up a level - - if verbose: - print( - "Tried directories %s but none started with prefix %s" - % (str(rootdirs), parentdir_prefix) - ) - raise NotThisMethod("rootdir doesn't start with parentdir_prefix") - - -@register_vcs_handler("git", "get_keywords") -def git_get_keywords(versionfile_abs): - """Extract version information from the given file.""" - # the code embedded in _version.py can just fetch the value of these - # keywords. When used from setup.py, we don't want to import _version.py, - # so we do it with a regexp instead. This function is not used from - # _version.py. - keywords = {} - try: - f = open(versionfile_abs) - for line in f.readlines(): - if line.strip().startswith("git_refnames ="): - mo = re.search(r'=\s*"(.*)"', line) - if mo: - keywords["refnames"] = mo.group(1) - if line.strip().startswith("git_full ="): - mo = re.search(r'=\s*"(.*)"', line) - if mo: - keywords["full"] = mo.group(1) - if line.strip().startswith("git_date ="): - mo = re.search(r'=\s*"(.*)"', line) - if mo: - keywords["date"] = mo.group(1) - f.close() - except OSError: - pass - return keywords - - -@register_vcs_handler("git", "keywords") -def git_versions_from_keywords(keywords, tag_prefix, verbose): - """Get version information from git keywords.""" - if not keywords: - raise NotThisMethod("no keywords at all, weird") - date = keywords.get("date") - if date is not None: - # git-2.2.0 added "%cI", which expands to an ISO-8601 -compliant - # datestamp. However we prefer "%ci" (which expands to an "ISO-8601 - # -like" string, which we must then edit to make compliant), because - # it's been around since git-1.5.3, and it's too difficult to - # discover which version we're using, or to work around using an - # older one. - date = date.strip().replace(" ", "T", 1).replace(" ", "", 1) - refnames = keywords["refnames"].strip() - if refnames.startswith("$Format"): - if verbose: - print("keywords are unexpanded, not using") - raise NotThisMethod("unexpanded keywords, not a git-archive tarball") - refs = {r.strip() for r in refnames.strip("()").split(",")} - # starting in git-1.8.3, tags are listed as "tag: foo-1.0" instead of - # just "foo-1.0". If we see a "tag: " prefix, prefer those. - TAG = "tag: " - tags = {r[len(TAG) :] for r in refs if r.startswith(TAG)} - if not tags: - # Either we're using git < 1.8.3, or there really are no tags. We use - # a heuristic: assume all version tags have a digit. The old git %d - # expansion behaves like git log --decorate=short and strips out the - # refs/heads/ and refs/tags/ prefixes that would let us distinguish - # between branches and tags. By ignoring refnames without digits, we - # filter out many common branch names like "release" and - # "stabilization", as well as "HEAD" and "master". - tags = {r for r in refs if re.search(r"\d", r)} - if verbose: - print("discarding '%s', no digits" % ",".join(refs - tags)) - if verbose: - print("likely tags: %s" % ",".join(sorted(tags))) - for ref in sorted(tags): - # sorting will prefer e.g. "2.0" over "2.0rc1" - if ref.startswith(tag_prefix): - r = ref[len(tag_prefix) :] - if verbose: - print("picking %s" % r) - return { - "version": r, - "full-revisionid": keywords["full"].strip(), - "dirty": False, - "error": None, - "date": date, - } - # no suitable tags, so version is "0+unknown", but full hex is still there - if verbose: - print("no suitable tags, using unknown + full revision id") - return { - "version": "0+unknown", - "full-revisionid": keywords["full"].strip(), - "dirty": False, - "error": "no suitable tags", - "date": None, - } - - -@register_vcs_handler("git", "pieces_from_vcs") -def git_pieces_from_vcs(tag_prefix, root, verbose, run_command=run_command): - """Get version from 'git describe' in the root of the source tree. - - This only gets called if the git-archive 'subst' keywords were *not* - expanded, and _version.py hasn't already been rewritten with a short - version string, meaning we're inside a checked out source tree. - """ - GITS = ["git"] - if sys.platform == "win32": - GITS = ["git.cmd", "git.exe"] - - out, rc = run_command( - GITS, ["rev-parse", "--git-dir"], cwd=root, hide_stderr=True - ) - if rc != 0: - if verbose: - print("Directory %s not under git control" % root) - raise NotThisMethod("'git rev-parse --git-dir' returned error") - - # if there is a tag matching tag_prefix, this yields TAG-NUM-gHEX[-dirty] - # if there isn't one, this yields HEX[-dirty] (no NUM) - describe_out, rc = run_command( - GITS, - [ - "describe", - "--tags", - "--dirty", - "--always", - "--long", - "--match", - "%s*" % tag_prefix, - ], - cwd=root, - ) - # --long was added in git-1.5.5 - if describe_out is None: - raise NotThisMethod("'git describe' failed") - describe_out = describe_out.strip() - full_out, rc = run_command(GITS, ["rev-parse", "HEAD"], cwd=root) - if full_out is None: - raise NotThisMethod("'git rev-parse' failed") - full_out = full_out.strip() - - pieces = {} - pieces["long"] = full_out - pieces["short"] = full_out[:7] # maybe improved later - pieces["error"] = None - - # parse describe_out. It will be like TAG-NUM-gHEX[-dirty] or HEX[-dirty] - # TAG might have hyphens. - git_describe = describe_out - - # look for -dirty suffix - dirty = git_describe.endswith("-dirty") - pieces["dirty"] = dirty - if dirty: - git_describe = git_describe[: git_describe.rindex("-dirty")] - - # now we have TAG-NUM-gHEX or HEX - - if "-" in git_describe: - # TAG-NUM-gHEX - mo = re.search(r"^(.+)-(\d+)-g([0-9a-f]+)$", git_describe) - if not mo: - # unparseable. Maybe git-describe is misbehaving? - pieces["error"] = ( - "unable to parse git-describe output: '%s'" % describe_out - ) - return pieces - - # tag - full_tag = mo.group(1) - if not full_tag.startswith(tag_prefix): - if verbose: - fmt = "tag '%s' doesn't start with prefix '%s'" - print(fmt % (full_tag, tag_prefix)) - pieces[ - "error" - ] = f"tag '{full_tag}' doesn't start with prefix '{tag_prefix}'" - return pieces - pieces["closest-tag"] = full_tag[len(tag_prefix) :] - - # distance: number of commits since tag - pieces["distance"] = int(mo.group(2)) - - # commit: short hex revision ID - pieces["short"] = mo.group(3) - - else: - # HEX: no tags - pieces["closest-tag"] = None - count_out, rc = run_command( - GITS, ["rev-list", "HEAD", "--count"], cwd=root - ) - pieces["distance"] = int(count_out) # total number of commits - - # commit date: see ISO-8601 comment in git_versions_from_keywords() - date = run_command(GITS, ["show", "-s", "--format=%ci", "HEAD"], cwd=root)[ - 0 - ].strip() - pieces["date"] = date.strip().replace(" ", "T", 1).replace(" ", "", 1) - - return pieces - - -def plus_or_dot(pieces): - """Return a + if we don't already have one, else return a .""" - if "+" in pieces.get("closest-tag", ""): - return "." - return "+" - - -def render_pep440(pieces): - """Build up version string, with post-release "local version identifier". - - Our goal: TAG[+DISTANCE.gHEX[.dirty]] . Note that if you - get a tagged build and then dirty it, you'll get TAG+0.gHEX.dirty - - Exceptions: - 1: no tags. git_describe was just HEX. 0+untagged.DISTANCE.gHEX[.dirty] - """ - if pieces["closest-tag"]: - rendered = pieces["closest-tag"] - if pieces["distance"] or pieces["dirty"]: - rendered += plus_or_dot(pieces) - rendered += "%d.g%s" % (pieces["distance"], pieces["short"]) - if pieces["dirty"]: - rendered += ".dirty" - else: - # exception #1 - rendered = "0+untagged.%d.g%s" % (pieces["distance"], pieces["short"]) - if pieces["dirty"]: - rendered += ".dirty" - return rendered - - -def render_pep440_pre(pieces): - """TAG[.post.devDISTANCE] -- No -dirty. - - Exceptions: - 1: no tags. 0.post.devDISTANCE - """ - if pieces["closest-tag"]: - rendered = pieces["closest-tag"] - if pieces["distance"]: - rendered += ".post.dev%d" % pieces["distance"] - else: - # exception #1 - rendered = "0.post.dev%d" % pieces["distance"] - return rendered - - -def render_pep440_post(pieces): - """TAG[.postDISTANCE[.dev0]+gHEX] . - - The ".dev0" means dirty. Note that .dev0 sorts backwards - (a dirty tree will appear "older" than the corresponding clean one), - but you shouldn't be releasing software with -dirty anyways. - - Exceptions: - 1: no tags. 0.postDISTANCE[.dev0] - """ - if pieces["closest-tag"]: - rendered = pieces["closest-tag"] - if pieces["distance"] or pieces["dirty"]: - rendered += ".post%d" % pieces["distance"] - if pieces["dirty"]: - rendered += ".dev0" - rendered += plus_or_dot(pieces) - rendered += "g%s" % pieces["short"] - else: - # exception #1 - rendered = "0.post%d" % pieces["distance"] - if pieces["dirty"]: - rendered += ".dev0" - rendered += "+g%s" % pieces["short"] - return rendered - - -def render_pep440_old(pieces): - """TAG[.postDISTANCE[.dev0]] . - - The ".dev0" means dirty. - - Exceptions: - 1: no tags. 0.postDISTANCE[.dev0] - """ - if pieces["closest-tag"]: - rendered = pieces["closest-tag"] - if pieces["distance"] or pieces["dirty"]: - rendered += ".post%d" % pieces["distance"] - if pieces["dirty"]: - rendered += ".dev0" - else: - # exception #1 - rendered = "0.post%d" % pieces["distance"] - if pieces["dirty"]: - rendered += ".dev0" - return rendered - - -def render_git_describe(pieces): - """TAG[-DISTANCE-gHEX][-dirty]. - - Like 'git describe --tags --dirty --always'. - - Exceptions: - 1: no tags. HEX[-dirty] (note: no 'g' prefix) - """ - if pieces["closest-tag"]: - rendered = pieces["closest-tag"] - if pieces["distance"]: - rendered += "-%d-g%s" % (pieces["distance"], pieces["short"]) - else: - # exception #1 - rendered = pieces["short"] - if pieces["dirty"]: - rendered += "-dirty" - return rendered - - -def render_git_describe_long(pieces): - """TAG-DISTANCE-gHEX[-dirty]. - - Like 'git describe --tags --dirty --always -long'. - The distance/hash is unconditional. - - Exceptions: - 1: no tags. HEX[-dirty] (note: no 'g' prefix) - """ - if pieces["closest-tag"]: - rendered = pieces["closest-tag"] - rendered += "-%d-g%s" % (pieces["distance"], pieces["short"]) - else: - # exception #1 - rendered = pieces["short"] - if pieces["dirty"]: - rendered += "-dirty" - return rendered - - -def render(pieces, style): - """Render the given version pieces into the requested style.""" - if pieces["error"]: - return { - "version": "unknown", - "full-revisionid": pieces.get("long"), - "dirty": None, - "error": pieces["error"], - "date": None, - } - - if not style or style == "default": - style = "pep440" # the default - - if style == "pep440": - rendered = render_pep440(pieces) - elif style == "pep440-pre": - rendered = render_pep440_pre(pieces) - elif style == "pep440-post": - rendered = render_pep440_post(pieces) - elif style == "pep440-old": - rendered = render_pep440_old(pieces) - elif style == "git-describe": - rendered = render_git_describe(pieces) - elif style == "git-describe-long": - rendered = render_git_describe_long(pieces) - else: - raise ValueError("unknown style '%s'" % style) - - return { - "version": rendered, - "full-revisionid": pieces["long"], - "dirty": pieces["dirty"], - "error": None, - "date": pieces.get("date"), - } - - -def get_versions(): - """Get version information or return default if unable to do so.""" - # I am in _version.py, which lives at ROOT/VERSIONFILE_SOURCE. If we have - # __file__, we can work backwards from there to the root. Some - # py2exe/bbfreeze/non-CPython implementations don't do __file__, in which - # case we can only use expanded keywords. - - cfg = get_config() - verbose = cfg.verbose - - try: - return git_versions_from_keywords( - get_keywords(), cfg.tag_prefix, verbose - ) - except NotThisMethod: - pass - - try: - root = os.path.realpath(__file__) - # versionfile_source is the relative path from the top of the source - # tree (where the .git directory might live) to this file. Invert - # this to find the root from __file__. - for i in cfg.versionfile_source.split("/"): - root = os.path.dirname(root) - except NameError: - return { - "version": "0+unknown", - "full-revisionid": None, - "dirty": None, - "error": "unable to find root of source tree", - "date": None, - } - - try: - pieces = git_pieces_from_vcs(cfg.tag_prefix, root, verbose) - return render(pieces, cfg.style) - except NotThisMethod: - pass - - try: - if cfg.parentdir_prefix: - return versions_from_parentdir(cfg.parentdir_prefix, root, verbose) - except NotThisMethod: - pass - - return { - "version": "0+unknown", - "full-revisionid": None, - "dirty": None, - "error": "unable to compute version", - "date": None, - } diff --git a/python/dask_cudf/setup.cfg b/python/dask_cudf/setup.cfg index f45bdf00430..66f4b8891d0 100644 --- a/python/dask_cudf/setup.cfg +++ b/python/dask_cudf/setup.cfg @@ -1,12 +1,4 @@ -# Copyright (c) 2020-2022, NVIDIA CORPORATION. - -[versioneer] -VCS = git -style = pep440 -versionfile_source = dask_cudf/_version.py -versionfile_build = dask_cudf/_version.py -tag_prefix = -parentdir_prefix = dask_cudf- +# Copyright (c) 2020-2023, NVIDIA CORPORATION. [isort] line_length=79 diff --git a/python/dask_cudf/setup.py b/python/dask_cudf/setup.py index 04145d23978..8611f2379f7 100644 --- a/python/dask_cudf/setup.py +++ b/python/dask_cudf/setup.py @@ -1,19 +1,14 @@ # Copyright (c) 2019-2023, NVIDIA CORPORATION. -import os - -import versioneer from setuptools import find_packages, setup -cuda_suffix = os.getenv("RAPIDS_PY_WHEEL_CUDA_SUFFIX", default="") - install_requires = [ "dask>=2023.1.1", "distributed>=2023.1.1", "fsspec>=0.6.0", "numpy", "pandas>=1.0,<1.6.0dev0", - f"cudf{cuda_suffix}==23.4.*", + "cudf==23.4.*", "cupy-cuda11x", ] @@ -27,21 +22,9 @@ ] } -if "RAPIDS_PY_WHEEL_VERSIONEER_OVERRIDE" in os.environ: - orig_get_versions = versioneer.get_versions - - version_override = os.environ["RAPIDS_PY_WHEEL_VERSIONEER_OVERRIDE"] - - def get_versions(): - data = orig_get_versions() - data["version"] = version_override - return data - - versioneer.get_versions = get_versions - setup( - name=f"dask-cudf{cuda_suffix}", - version=versioneer.get_version(), + name="dask-cudf", + version="23.04.00", description="Utilities for Dask and cuDF interactions", url="https://github.com/rapidsai/cudf", author="NVIDIA Corporation", @@ -57,7 +40,6 @@ def get_versions(): "Programming Language :: Python :: 3.10", ], packages=find_packages(exclude=["tests", "tests.*"]), - cmdclass=versioneer.get_cmdclass(), install_requires=install_requires, extras_require=extras_require, ) diff --git a/python/dask_cudf/versioneer.py b/python/dask_cudf/versioneer.py deleted file mode 100644 index a560f2e8797..00000000000 --- a/python/dask_cudf/versioneer.py +++ /dev/null @@ -1,1904 +0,0 @@ -# Version: 0.18 - -"""The Versioneer - like a rocketeer, but for versions. - -The Versioneer -============== - -* like a rocketeer, but for versions! -* https://github.com/warner/python-versioneer -* Brian Warner -* License: Public Domain -* Compatible With: python2.6, 2.7, 3.2, 3.3, 3.4, 3.5, 3.6, and pypy -* [![Latest Version] -(https://pypip.in/version/versioneer/badge.svg?style=flat) -](https://pypi.python.org/pypi/versioneer/) -* [![Build Status] -(https://travis-ci.org/warner/python-versioneer.png?branch=master) -](https://travis-ci.org/warner/python-versioneer) - -This is a tool for managing a recorded version number in distutils-based -python projects. The goal is to remove the tedious and error-prone "update -the embedded version string" step from your release process. Making a new -release should be as easy as recording a new tag in your version-control -system, and maybe making new tarballs. - - -## Quick Install - -* `pip install versioneer` to somewhere to your $PATH -* add a `[versioneer]` section to your setup.cfg (see below) -* run `versioneer install` in your source tree, commit the results - -## Version Identifiers - -Source trees come from a variety of places: - -* a version-control system checkout (mostly used by developers) -* a nightly tarball, produced by build automation -* a snapshot tarball, produced by a web-based VCS browser, like github's - "tarball from tag" feature -* a release tarball, produced by "setup.py sdist", distributed through PyPI - -Within each source tree, the version identifier (either a string or a number, -this tool is format-agnostic) can come from a variety of places: - -* ask the VCS tool itself, e.g. "git describe" (for checkouts), which knows - about recent "tags" and an absolute revision-id -* the name of the directory into which the tarball was unpacked -* an expanded VCS keyword ($Id$, etc) -* a `_version.py` created by some earlier build step - -For released software, the version identifier is closely related to a VCS -tag. Some projects use tag names that include more than just the version -string (e.g. "myproject-1.2" instead of just "1.2"), in which case the tool -needs to strip the tag prefix to extract the version identifier. For -unreleased software (between tags), the version identifier should provide -enough information to help developers recreate the same tree, while also -giving them an idea of roughly how old the tree is (after version 1.2, before -version 1.3). Many VCS systems can report a description that captures this, -for example `git describe --tags --dirty --always` reports things like -"0.7-1-g574ab98-dirty" to indicate that the checkout is one revision past the -0.7 tag, has a unique revision id of "574ab98", and is "dirty" (it has -uncommitted changes. - -The version identifier is used for multiple purposes: - -* to allow the module to self-identify its version: `myproject.__version__` -* to choose a name and prefix for a 'setup.py sdist' tarball - -## Theory of Operation - -Versioneer works by adding a special `_version.py` file into your source -tree, where your `__init__.py` can import it. This `_version.py` knows how to -dynamically ask the VCS tool for version information at import time. - -`_version.py` also contains `$Revision$` markers, and the installation -process marks `_version.py` to have this marker rewritten with a tag name -during the `git archive` command. As a result, generated tarballs will -contain enough information to get the proper version. - -To allow `setup.py` to compute a version too, a `versioneer.py` is added to -the top level of your source tree, next to `setup.py` and the `setup.cfg` -that configures it. This overrides several distutils/setuptools commands to -compute the version when invoked, and changes `setup.py build` and `setup.py -sdist` to replace `_version.py` with a small static file that contains just -the generated version data. - -## Installation - -See [INSTALL.md](./INSTALL.md) for detailed installation instructions. - -## Version-String Flavors - -Code which uses Versioneer can learn about its version string at runtime by -importing `_version` from your main `__init__.py` file and running the -`get_versions()` function. From the "outside" (e.g. in `setup.py`), you can -import the top-level `versioneer.py` and run `get_versions()`. - -Both functions return a dictionary with different flavors of version -information: - -* `['version']`: A condensed version string, rendered using the selected - style. This is the most commonly used value for the project's version - string. The default "pep440" style yields strings like `0.11`, - `0.11+2.g1076c97`, or `0.11+2.g1076c97.dirty`. See the "Styles" section - below for alternative styles. - -* `['full-revisionid']`: detailed revision identifier. For Git, this is the - full SHA1 commit id, e.g. "1076c978a8d3cfc70f408fe5974aa6c092c949ac". - -* `['date']`: Date and time of the latest `HEAD` commit. For Git, it is the - commit date in ISO 8601 format. This will be None if the date is not - available. - -* `['dirty']`: a boolean, True if the tree has uncommitted changes. Note that - this is only accurate if run in a VCS checkout, otherwise it is likely to - be False or None - -* `['error']`: if the version string could not be computed, this will be set - to a string describing the problem, otherwise it will be None. It may be - useful to throw an exception in setup.py if this is set, to avoid e.g. - creating tarballs with a version string of "unknown". - -Some variants are more useful than others. Including `full-revisionid` in a -bug report should allow developers to reconstruct the exact code being tested -(or indicate the presence of local changes that should be shared with the -developers). `version` is suitable for display in an "about" box or a CLI -`--version` output: it can be easily compared against release notes and lists -of bugs fixed in various releases. - -The installer adds the following text to your `__init__.py` to place a basic -version in `YOURPROJECT.__version__`: - - from ._version import get_versions - __version__ = get_versions()['version'] - del get_versions - -## Styles - -The setup.cfg `style=` configuration controls how the VCS information is -rendered into a version string. - -The default style, "pep440", produces a PEP440-compliant string, equal to the -un-prefixed tag name for actual releases, and containing an additional "local -version" section with more detail for in-between builds. For Git, this is -TAG[+DISTANCE.gHEX[.dirty]] , using information from `git describe --tags ---dirty --always`. For example "0.11+2.g1076c97.dirty" indicates that the -tree is like the "1076c97" commit but has uncommitted changes (".dirty"), and -that this commit is two revisions ("+2") beyond the "0.11" tag. For released -software (exactly equal to a known tag), the identifier will only contain the -stripped tag, e.g. "0.11". - -Other styles are available. See [details.md](details.md) in the Versioneer -source tree for descriptions. - -## Debugging - -Versioneer tries to avoid fatal errors: if something goes wrong, it will tend -to return a version of "0+unknown". To investigate the problem, run `setup.py -version`, which will run the version-lookup code in a verbose mode, and will -display the full contents of `get_versions()` (including the `error` string, -which may help identify what went wrong). - -## Known Limitations - -Some situations are known to cause problems for Versioneer. This details the -most significant ones. More can be found on Github -[issues page](https://github.com/warner/python-versioneer/issues). - -### Subprojects - -Versioneer has limited support for source trees in which `setup.py` is not in -the root directory (e.g. `setup.py` and `.git/` are *not* siblings). The are -two common reasons why `setup.py` might not be in the root: - -* Source trees which contain multiple subprojects, such as - [Buildbot](https://github.com/buildbot/buildbot), which contains both - "master" and "slave" subprojects, each with their own `setup.py`, - `setup.cfg`, and `tox.ini`. Projects like these produce multiple PyPI - distributions (and upload multiple independently-installable tarballs). -* Source trees whose main purpose is to contain a C library, but which also - provide bindings to Python (and perhaps other langauges) in subdirectories. - -Versioneer will look for `.git` in parent directories, and most operations -should get the right version string. However `pip` and `setuptools` have bugs -and implementation details which frequently cause `pip install .` from a -subproject directory to fail to find a correct version string (so it usually -defaults to `0+unknown`). - -`pip install --editable .` should work correctly. `setup.py install` might -work too. - -Pip-8.1.1 is known to have this problem, but hopefully it will get fixed in -some later version. - -[Bug #38](https://github.com/warner/python-versioneer/issues/38) is tracking -this issue. The discussion in -[PR #61](https://github.com/warner/python-versioneer/pull/61) describes the -issue from the Versioneer side in more detail. -[pip PR#3176](https://github.com/pypa/pip/pull/3176) and -[pip PR#3615](https://github.com/pypa/pip/pull/3615) contain work to improve -pip to let Versioneer work correctly. - -Versioneer-0.16 and earlier only looked for a `.git` directory next to the -`setup.cfg`, so subprojects were completely unsupported with those releases. - -### Editable installs with setuptools <= 18.5 - -`setup.py develop` and `pip install --editable .` allow you to install a -project into a virtualenv once, then continue editing the source code (and -test) without re-installing after every change. - -"Entry-point scripts" (`setup(entry_points={"console_scripts": ..})`) are a -convenient way to specify executable scripts that should be installed along -with the python package. - -These both work as expected when using modern setuptools. When using -setuptools-18.5 or earlier, however, certain operations will cause -`pkg_resources.DistributionNotFound` errors when running the entrypoint -script, which must be resolved by re-installing the package. This happens -when the install happens with one version, then the egg_info data is -regenerated while a different version is checked out. Many setup.py commands -cause egg_info to be rebuilt (including `sdist`, `wheel`, and installing into -a different virtualenv), so this can be surprising. - -[Bug #83](https://github.com/warner/python-versioneer/issues/83) describes -this one, but upgrading to a newer version of setuptools should probably -resolve it. - -### Unicode version strings - -While Versioneer works (and is continually tested) with both Python 2 and -Python 3, it is not entirely consistent with bytes-vs-unicode distinctions. -Newer releases probably generate unicode version strings on py2. It's not -clear that this is wrong, but it may be surprising for applications when then -write these strings to a network connection or include them in bytes-oriented -APIs like cryptographic checksums. - -[Bug #71](https://github.com/warner/python-versioneer/issues/71) investigates -this question. - - -## Updating Versioneer - -To upgrade your project to a new release of Versioneer, do the following: - -* install the new Versioneer (`pip install -U versioneer` or equivalent) -* edit `setup.cfg`, if necessary, to include any new configuration settings - indicated by the release notes. See [UPGRADING](./UPGRADING.md) for details. -* re-run `versioneer install` in your source tree, to replace - `SRC/_version.py` -* commit any changed files - -## Future Directions - -This tool is designed to make it easily extended to other version-control -systems: all VCS-specific components are in separate directories like -src/git/ . The top-level `versioneer.py` script is assembled from these -components by running make-versioneer.py . In the future, make-versioneer.py -will take a VCS name as an argument, and will construct a version of -`versioneer.py` that is specific to the given VCS. It might also take the -configuration arguments that are currently provided manually during -installation by editing setup.py . Alternatively, it might go the other -direction and include code from all supported VCS systems, reducing the -number of intermediate scripts. - - -## License - -To make Versioneer easier to embed, all its code is dedicated to the public -domain. The `_version.py` that it creates is also in the public domain. -Specifically, both are released under the Creative Commons "Public Domain -Dedication" license (CC0-1.0), as described in -https://creativecommons.org/publicdomain/zero/1.0/ . - -""" - -from __future__ import print_function - -import errno -import json -import os -import re -import subprocess -import sys - -try: - import configparser -except ImportError: - import ConfigParser as configparser - - -class VersioneerConfig: - """Container for Versioneer configuration parameters.""" - - -def get_root(): - """Get the project root directory. - - We require that all commands are run from the project root, i.e. the - directory that contains setup.py, setup.cfg, and versioneer.py . - """ - root = os.path.realpath(os.path.abspath(os.getcwd())) - setup_py = os.path.join(root, "setup.py") - versioneer_py = os.path.join(root, "versioneer.py") - if not (os.path.exists(setup_py) or os.path.exists(versioneer_py)): - # allow 'python path/to/setup.py COMMAND' - root = os.path.dirname(os.path.realpath(os.path.abspath(sys.argv[0]))) - setup_py = os.path.join(root, "setup.py") - versioneer_py = os.path.join(root, "versioneer.py") - if not (os.path.exists(setup_py) or os.path.exists(versioneer_py)): - err = ( - "Versioneer was unable to run the project root directory. " - "Versioneer requires setup.py to be executed from " - "its immediate directory (like 'python setup.py COMMAND'), " - "or in a way that lets it use sys.argv[0] to find the root " - "(like 'python path/to/setup.py COMMAND')." - ) - raise VersioneerBadRootError(err) - try: - # Certain runtime workflows (setup.py install/develop in a setuptools - # tree) execute all dependencies in a single python process, so - # "versioneer" may be imported multiple times, and python's shared - # module-import table will cache the first one. So we can't use - # os.path.dirname(__file__), as that will find whichever - # versioneer.py was first imported, even in later projects. - me = os.path.realpath(os.path.abspath(__file__)) - me_dir = os.path.normcase(os.path.splitext(me)[0]) - vsr_dir = os.path.normcase(os.path.splitext(versioneer_py)[0]) - if me_dir != vsr_dir: - print( - "Warning: build in %s is using versioneer.py from %s" - % (os.path.dirname(me), versioneer_py) - ) - except NameError: - pass - return root - - -def get_config_from_root(root): - """Read the project setup.cfg file to determine Versioneer config.""" - # This might raise EnvironmentError (if setup.cfg is missing), or - # configparser.NoSectionError (if it lacks a [versioneer] section), or - # configparser.NoOptionError (if it lacks "VCS="). See the docstring at - # the top of versioneer.py for instructions on writing your setup.cfg . - setup_cfg = os.path.join(root, "setup.cfg") - parser = configparser.SafeConfigParser() - with open(setup_cfg, "r") as f: - parser.readfp(f) - VCS = parser.get("versioneer", "VCS") # mandatory - - def get(parser, name): - if parser.has_option("versioneer", name): - return parser.get("versioneer", name) - return None - - cfg = VersioneerConfig() - cfg.VCS = VCS - cfg.style = get(parser, "style") or "" - cfg.versionfile_source = get(parser, "versionfile_source") - cfg.versionfile_build = get(parser, "versionfile_build") - cfg.tag_prefix = get(parser, "tag_prefix") - if cfg.tag_prefix in ("''", '""'): - cfg.tag_prefix = "" - cfg.parentdir_prefix = get(parser, "parentdir_prefix") - cfg.verbose = get(parser, "verbose") - return cfg - - -class NotThisMethod(Exception): - """Exception raised if a method is not valid for the current scenario.""" - - -# these dictionaries contain VCS-specific tools -LONG_VERSION_PY = {} -HANDLERS = {} - - -def register_vcs_handler(vcs, method): # decorator - """Decorator to mark a method as the handler for a particular VCS.""" - - def decorate(f): - """Store f in HANDLERS[vcs][method].""" - if vcs not in HANDLERS: - HANDLERS[vcs] = {} - HANDLERS[vcs][method] = f - return f - - return decorate - - -def run_command( - commands, args, cwd=None, verbose=False, hide_stderr=False, env=None -): - """Call the given command(s).""" - assert isinstance(commands, list) - p = None - for c in commands: - try: - dispcmd = str([c] + args) - # remember shell=False, so use git.cmd on windows, not just git - p = subprocess.Popen( - [c] + args, - cwd=cwd, - env=env, - stdout=subprocess.PIPE, - stderr=(subprocess.PIPE if hide_stderr else None), - ) - break - except EnvironmentError: - e = sys.exc_info()[1] - if e.errno == errno.ENOENT: - continue - if verbose: - print("unable to run %s" % dispcmd) - print(e) - return None, None - else: - if verbose: - print("unable to find command, tried %s" % (commands,)) - return None, None - stdout = p.communicate()[0].strip() - if sys.version_info[0] >= 3: - stdout = stdout.decode() - if p.returncode != 0: - if verbose: - print("unable to run %s (error)" % dispcmd) - print("stdout was %s" % stdout) - return None, p.returncode - return stdout, p.returncode - - -LONG_VERSION_PY[ - "git" -] = r''' -# This file helps to compute a version number in source trees obtained from -# git-archive tarball (such as those provided by githubs download-from-tag -# feature). Distribution tarballs (built by setup.py sdist) and build -# directories (produced by setup.py build) will contain a much shorter file -# that just contains the computed version number. - -# This file is released into the public domain. Generated by -# versioneer-0.18 (https://github.com/warner/python-versioneer) - -"""Git implementation of _version.py.""" - -import errno -import os -import re -import subprocess -import sys - - -def get_keywords(): - """Get the keywords needed to look up the version information.""" - # these strings will be replaced by git during git-archive. - # setup.py/versioneer.py will grep for the variable names, so they must - # each be defined on a line of their own. _version.py will just call - # get_keywords(). - git_refnames = "%(DOLLAR)sFormat:%%d%(DOLLAR)s" - git_full = "%(DOLLAR)sFormat:%%H%(DOLLAR)s" - git_date = "%(DOLLAR)sFormat:%%ci%(DOLLAR)s" - keywords = {"refnames": git_refnames, "full": git_full, "date": git_date} - return keywords - - -class VersioneerConfig: - """Container for Versioneer configuration parameters.""" - - -def get_config(): - """Create, populate and return the VersioneerConfig() object.""" - # these strings are filled in when 'setup.py versioneer' creates - # _version.py - cfg = VersioneerConfig() - cfg.VCS = "git" - cfg.style = "%(STYLE)s" - cfg.tag_prefix = "%(TAG_PREFIX)s" - cfg.parentdir_prefix = "%(PARENTDIR_PREFIX)s" - cfg.versionfile_source = "%(VERSIONFILE_SOURCE)s" - cfg.verbose = False - return cfg - - -class NotThisMethod(Exception): - """Exception raised if a method is not valid for the current scenario.""" - - -LONG_VERSION_PY = {} -HANDLERS = {} - - -def register_vcs_handler(vcs, method): # decorator - """Decorator to mark a method as the handler for a particular VCS.""" - def decorate(f): - """Store f in HANDLERS[vcs][method].""" - if vcs not in HANDLERS: - HANDLERS[vcs] = {} - HANDLERS[vcs][method] = f - return f - return decorate - - -def run_command(commands, args, cwd=None, verbose=False, hide_stderr=False, - env=None): - """Call the given command(s).""" - assert isinstance(commands, list) - p = None - for c in commands: - try: - dispcmd = str([c] + args) - # remember shell=False, so use git.cmd on windows, not just git - p = subprocess.Popen([c] + args, cwd=cwd, env=env, - stdout=subprocess.PIPE, - stderr=(subprocess.PIPE if hide_stderr - else None)) - break - except EnvironmentError: - e = sys.exc_info()[1] - if e.errno == errno.ENOENT: - continue - if verbose: - print("unable to run %%s" %% dispcmd) - print(e) - return None, None - else: - if verbose: - print("unable to find command, tried %%s" %% (commands,)) - return None, None - stdout = p.communicate()[0].strip() - if sys.version_info[0] >= 3: - stdout = stdout.decode() - if p.returncode != 0: - if verbose: - print("unable to run %%s (error)" %% dispcmd) - print("stdout was %%s" %% stdout) - return None, p.returncode - return stdout, p.returncode - - -def versions_from_parentdir(parentdir_prefix, root, verbose): - """Try to determine the version from the parent directory name. - - Source tarballs conventionally unpack into a directory that includes both - the project name and a version string. We will also support searching up - two directory levels for an appropriately named parent directory - """ - rootdirs = [] - - for i in range(3): - dirname = os.path.basename(root) - if dirname.startswith(parentdir_prefix): - return {"version": dirname[len(parentdir_prefix):], - "full-revisionid": None, - "dirty": False, "error": None, "date": None} - else: - rootdirs.append(root) - root = os.path.dirname(root) # up a level - - if verbose: - print("Tried directories %%s but none started with prefix %%s" %% - (str(rootdirs), parentdir_prefix)) - raise NotThisMethod("rootdir doesn't start with parentdir_prefix") - - -@register_vcs_handler("git", "get_keywords") -def git_get_keywords(versionfile_abs): - """Extract version information from the given file.""" - # the code embedded in _version.py can just fetch the value of these - # keywords. When used from setup.py, we don't want to import _version.py, - # so we do it with a regexp instead. This function is not used from - # _version.py. - keywords = {} - try: - f = open(versionfile_abs, "r") - for line in f.readlines(): - if line.strip().startswith("git_refnames ="): - mo = re.search(r'=\s*"(.*)"', line) - if mo: - keywords["refnames"] = mo.group(1) - if line.strip().startswith("git_full ="): - mo = re.search(r'=\s*"(.*)"', line) - if mo: - keywords["full"] = mo.group(1) - if line.strip().startswith("git_date ="): - mo = re.search(r'=\s*"(.*)"', line) - if mo: - keywords["date"] = mo.group(1) - f.close() - except EnvironmentError: - pass - return keywords - - -@register_vcs_handler("git", "keywords") -def git_versions_from_keywords(keywords, tag_prefix, verbose): - """Get version information from git keywords.""" - if not keywords: - raise NotThisMethod("no keywords at all, weird") - date = keywords.get("date") - if date is not None: - # git-2.2.0 added "%%cI", which expands to an ISO-8601 -compliant - # datestamp. However we prefer "%%ci" (which expands to an "ISO-8601 - # -like" string, which we must then edit to make compliant), because - # it's been around since git-1.5.3, and it's too difficult to - # discover which version we're using, or to work around using an - # older one. - date = date.strip().replace(" ", "T", 1).replace(" ", "", 1) - refnames = keywords["refnames"].strip() - if refnames.startswith("$Format"): - if verbose: - print("keywords are unexpanded, not using") - raise NotThisMethod("unexpanded keywords, not a git-archive tarball") - refs = set([r.strip() for r in refnames.strip("()").split(",")]) - # starting in git-1.8.3, tags are listed as "tag: foo-1.0" instead of - # just "foo-1.0". If we see a "tag: " prefix, prefer those. - TAG = "tag: " - tags = set([r[len(TAG):] for r in refs if r.startswith(TAG)]) - if not tags: - # Either we're using git < 1.8.3, or there really are no tags. We use - # a heuristic: assume all version tags have a digit. The old git %%d - # expansion behaves like git log --decorate=short and strips out the - # refs/heads/ and refs/tags/ prefixes that would let us distinguish - # between branches and tags. By ignoring refnames without digits, we - # filter out many common branch names like "release" and - # "stabilization", as well as "HEAD" and "master". - tags = set([r for r in refs if re.search(r'\d', r)]) - if verbose: - print("discarding '%%s', no digits" %% ",".join(refs - tags)) - if verbose: - print("likely tags: %%s" %% ",".join(sorted(tags))) - for ref in sorted(tags): - # sorting will prefer e.g. "2.0" over "2.0rc1" - if ref.startswith(tag_prefix): - r = ref[len(tag_prefix):] - if verbose: - print("picking %%s" %% r) - return {"version": r, - "full-revisionid": keywords["full"].strip(), - "dirty": False, "error": None, - "date": date} - # no suitable tags, so version is "0+unknown", but full hex is still there - if verbose: - print("no suitable tags, using unknown + full revision id") - return {"version": "0+unknown", - "full-revisionid": keywords["full"].strip(), - "dirty": False, "error": "no suitable tags", "date": None} - - -@register_vcs_handler("git", "pieces_from_vcs") -def git_pieces_from_vcs(tag_prefix, root, verbose, run_command=run_command): - """Get version from 'git describe' in the root of the source tree. - - This only gets called if the git-archive 'subst' keywords were *not* - expanded, and _version.py hasn't already been rewritten with a short - version string, meaning we're inside a checked out source tree. - """ - GITS = ["git"] - if sys.platform == "win32": - GITS = ["git.cmd", "git.exe"] - - out, rc = run_command(GITS, ["rev-parse", "--git-dir"], cwd=root, - hide_stderr=True) - if rc != 0: - if verbose: - print("Directory %%s not under git control" %% root) - raise NotThisMethod("'git rev-parse --git-dir' returned error") - - # if there is a tag matching tag_prefix, this yields TAG-NUM-gHEX[-dirty] - # if there isn't one, this yields HEX[-dirty] (no NUM) - describe_out, rc = run_command(GITS, ["describe", "--tags", "--dirty", - "--always", "--long", - "--match", "%%s*" %% tag_prefix], - cwd=root) - # --long was added in git-1.5.5 - if describe_out is None: - raise NotThisMethod("'git describe' failed") - describe_out = describe_out.strip() - full_out, rc = run_command(GITS, ["rev-parse", "HEAD"], cwd=root) - if full_out is None: - raise NotThisMethod("'git rev-parse' failed") - full_out = full_out.strip() - - pieces = {} - pieces["long"] = full_out - pieces["short"] = full_out[:7] # maybe improved later - pieces["error"] = None - - # parse describe_out. It will be like TAG-NUM-gHEX[-dirty] or HEX[-dirty] - # TAG might have hyphens. - git_describe = describe_out - - # look for -dirty suffix - dirty = git_describe.endswith("-dirty") - pieces["dirty"] = dirty - if dirty: - git_describe = git_describe[:git_describe.rindex("-dirty")] - - # now we have TAG-NUM-gHEX or HEX - - if "-" in git_describe: - # TAG-NUM-gHEX - mo = re.search(r'^(.+)-(\d+)-g([0-9a-f]+)$', git_describe) - if not mo: - # unparseable. Maybe git-describe is misbehaving? - pieces["error"] = ("unable to parse git-describe output: '%%s'" - %% describe_out) - return pieces - - # tag - full_tag = mo.group(1) - if not full_tag.startswith(tag_prefix): - if verbose: - fmt = "tag '%%s' doesn't start with prefix '%%s'" - print(fmt %% (full_tag, tag_prefix)) - pieces["error"] = ("tag '%%s' doesn't start with prefix '%%s'" - %% (full_tag, tag_prefix)) - return pieces - pieces["closest-tag"] = full_tag[len(tag_prefix):] - - # distance: number of commits since tag - pieces["distance"] = int(mo.group(2)) - - # commit: short hex revision ID - pieces["short"] = mo.group(3) - - else: - # HEX: no tags - pieces["closest-tag"] = None - count_out, rc = run_command(GITS, ["rev-list", "HEAD", "--count"], - cwd=root) - pieces["distance"] = int(count_out) # total number of commits - - # commit date: see ISO-8601 comment in git_versions_from_keywords() - date = run_command(GITS, ["show", "-s", "--format=%%ci", "HEAD"], - cwd=root)[0].strip() - pieces["date"] = date.strip().replace(" ", "T", 1).replace(" ", "", 1) - - return pieces - - -def plus_or_dot(pieces): - """Return a + if we don't already have one, else return a .""" - if "+" in pieces.get("closest-tag", ""): - return "." - return "+" - - -def render_pep440(pieces): - """Build up version string, with post-release "local version identifier". - - Our goal: TAG[+DISTANCE.gHEX[.dirty]] . Note that if you - get a tagged build and then dirty it, you'll get TAG+0.gHEX.dirty - - Exceptions: - 1: no tags. git_describe was just HEX. 0+untagged.DISTANCE.gHEX[.dirty] - """ - if pieces["closest-tag"]: - rendered = pieces["closest-tag"] - if pieces["distance"] or pieces["dirty"]: - rendered += plus_or_dot(pieces) - rendered += "%%d.g%%s" %% (pieces["distance"], pieces["short"]) - if pieces["dirty"]: - rendered += ".dirty" - else: - # exception #1 - rendered = "0+untagged.%%d.g%%s" %% (pieces["distance"], - pieces["short"]) - if pieces["dirty"]: - rendered += ".dirty" - return rendered - - -def render_pep440_pre(pieces): - """TAG[.post.devDISTANCE] -- No -dirty. - - Exceptions: - 1: no tags. 0.post.devDISTANCE - """ - if pieces["closest-tag"]: - rendered = pieces["closest-tag"] - if pieces["distance"]: - rendered += ".post.dev%%d" %% pieces["distance"] - else: - # exception #1 - rendered = "0.post.dev%%d" %% pieces["distance"] - return rendered - - -def render_pep440_post(pieces): - """TAG[.postDISTANCE[.dev0]+gHEX] . - - The ".dev0" means dirty. Note that .dev0 sorts backwards - (a dirty tree will appear "older" than the corresponding clean one), - but you shouldn't be releasing software with -dirty anyways. - - Exceptions: - 1: no tags. 0.postDISTANCE[.dev0] - """ - if pieces["closest-tag"]: - rendered = pieces["closest-tag"] - if pieces["distance"] or pieces["dirty"]: - rendered += ".post%%d" %% pieces["distance"] - if pieces["dirty"]: - rendered += ".dev0" - rendered += plus_or_dot(pieces) - rendered += "g%%s" %% pieces["short"] - else: - # exception #1 - rendered = "0.post%%d" %% pieces["distance"] - if pieces["dirty"]: - rendered += ".dev0" - rendered += "+g%%s" %% pieces["short"] - return rendered - - -def render_pep440_old(pieces): - """TAG[.postDISTANCE[.dev0]] . - - The ".dev0" means dirty. - - Exceptions: - 1: no tags. 0.postDISTANCE[.dev0] - """ - if pieces["closest-tag"]: - rendered = pieces["closest-tag"] - if pieces["distance"] or pieces["dirty"]: - rendered += ".post%%d" %% pieces["distance"] - if pieces["dirty"]: - rendered += ".dev0" - else: - # exception #1 - rendered = "0.post%%d" %% pieces["distance"] - if pieces["dirty"]: - rendered += ".dev0" - return rendered - - -def render_git_describe(pieces): - """TAG[-DISTANCE-gHEX][-dirty]. - - Like 'git describe --tags --dirty --always'. - - Exceptions: - 1: no tags. HEX[-dirty] (note: no 'g' prefix) - """ - if pieces["closest-tag"]: - rendered = pieces["closest-tag"] - if pieces["distance"]: - rendered += "-%%d-g%%s" %% (pieces["distance"], pieces["short"]) - else: - # exception #1 - rendered = pieces["short"] - if pieces["dirty"]: - rendered += "-dirty" - return rendered - - -def render_git_describe_long(pieces): - """TAG-DISTANCE-gHEX[-dirty]. - - Like 'git describe --tags --dirty --always -long'. - The distance/hash is unconditional. - - Exceptions: - 1: no tags. HEX[-dirty] (note: no 'g' prefix) - """ - if pieces["closest-tag"]: - rendered = pieces["closest-tag"] - rendered += "-%%d-g%%s" %% (pieces["distance"], pieces["short"]) - else: - # exception #1 - rendered = pieces["short"] - if pieces["dirty"]: - rendered += "-dirty" - return rendered - - -def render(pieces, style): - """Render the given version pieces into the requested style.""" - if pieces["error"]: - return {"version": "unknown", - "full-revisionid": pieces.get("long"), - "dirty": None, - "error": pieces["error"], - "date": None} - - if not style or style == "default": - style = "pep440" # the default - - if style == "pep440": - rendered = render_pep440(pieces) - elif style == "pep440-pre": - rendered = render_pep440_pre(pieces) - elif style == "pep440-post": - rendered = render_pep440_post(pieces) - elif style == "pep440-old": - rendered = render_pep440_old(pieces) - elif style == "git-describe": - rendered = render_git_describe(pieces) - elif style == "git-describe-long": - rendered = render_git_describe_long(pieces) - else: - raise ValueError("unknown style '%%s'" %% style) - - return {"version": rendered, "full-revisionid": pieces["long"], - "dirty": pieces["dirty"], "error": None, - "date": pieces.get("date")} - - -def get_versions(): - """Get version information or return default if unable to do so.""" - # I am in _version.py, which lives at ROOT/VERSIONFILE_SOURCE. If we have - # __file__, we can work backwards from there to the root. Some - # py2exe/bbfreeze/non-CPython implementations don't do __file__, in which - # case we can only use expanded keywords. - - cfg = get_config() - verbose = cfg.verbose - - try: - return git_versions_from_keywords(get_keywords(), cfg.tag_prefix, - verbose) - except NotThisMethod: - pass - - try: - root = os.path.realpath(__file__) - # versionfile_source is the relative path from the top of the source - # tree (where the .git directory might live) to this file. Invert - # this to find the root from __file__. - for i in cfg.versionfile_source.split('/'): - root = os.path.dirname(root) - except NameError: - return {"version": "0+unknown", "full-revisionid": None, - "dirty": None, - "error": "unable to find root of source tree", - "date": None} - - try: - pieces = git_pieces_from_vcs(cfg.tag_prefix, root, verbose) - return render(pieces, cfg.style) - except NotThisMethod: - pass - - try: - if cfg.parentdir_prefix: - return versions_from_parentdir(cfg.parentdir_prefix, root, verbose) - except NotThisMethod: - pass - - return {"version": "0+unknown", "full-revisionid": None, - "dirty": None, - "error": "unable to compute version", "date": None} -''' - - -@register_vcs_handler("git", "get_keywords") -def git_get_keywords(versionfile_abs): - """Extract version information from the given file.""" - # the code embedded in _version.py can just fetch the value of these - # keywords. When used from setup.py, we don't want to import _version.py, - # so we do it with a regexp instead. This function is not used from - # _version.py. - keywords = {} - try: - f = open(versionfile_abs, "r") - for line in f.readlines(): - if line.strip().startswith("git_refnames ="): - mo = re.search(r'=\s*"(.*)"', line) - if mo: - keywords["refnames"] = mo.group(1) - if line.strip().startswith("git_full ="): - mo = re.search(r'=\s*"(.*)"', line) - if mo: - keywords["full"] = mo.group(1) - if line.strip().startswith("git_date ="): - mo = re.search(r'=\s*"(.*)"', line) - if mo: - keywords["date"] = mo.group(1) - f.close() - except EnvironmentError: - pass - return keywords - - -@register_vcs_handler("git", "keywords") -def git_versions_from_keywords(keywords, tag_prefix, verbose): - """Get version information from git keywords.""" - if not keywords: - raise NotThisMethod("no keywords at all, weird") - date = keywords.get("date") - if date is not None: - # git-2.2.0 added "%cI", which expands to an ISO-8601 -compliant - # datestamp. However we prefer "%ci" (which expands to an "ISO-8601 - # -like" string, which we must then edit to make compliant), because - # it's been around since git-1.5.3, and it's too difficult to - # discover which version we're using, or to work around using an - # older one. - date = date.strip().replace(" ", "T", 1).replace(" ", "", 1) - refnames = keywords["refnames"].strip() - if refnames.startswith("$Format"): - if verbose: - print("keywords are unexpanded, not using") - raise NotThisMethod("unexpanded keywords, not a git-archive tarball") - refs = set([r.strip() for r in refnames.strip("()").split(",")]) - # starting in git-1.8.3, tags are listed as "tag: foo-1.0" instead of - # just "foo-1.0". If we see a "tag: " prefix, prefer those. - TAG = "tag: " - tags = set([r[len(TAG) :] for r in refs if r.startswith(TAG)]) - if not tags: - # Either we're using git < 1.8.3, or there really are no tags. We use - # a heuristic: assume all version tags have a digit. The old git %d - # expansion behaves like git log --decorate=short and strips out the - # refs/heads/ and refs/tags/ prefixes that would let us distinguish - # between branches and tags. By ignoring refnames without digits, we - # filter out many common branch names like "release" and - # "stabilization", as well as "HEAD" and "master". - tags = set([r for r in refs if re.search(r"\d", r)]) - if verbose: - print("discarding '%s', no digits" % ",".join(refs - tags)) - if verbose: - print("likely tags: %s" % ",".join(sorted(tags))) - for ref in sorted(tags): - # sorting will prefer e.g. "2.0" over "2.0rc1" - if ref.startswith(tag_prefix): - r = ref[len(tag_prefix) :] - if verbose: - print("picking %s" % r) - return { - "version": r, - "full-revisionid": keywords["full"].strip(), - "dirty": False, - "error": None, - "date": date, - } - # no suitable tags, so version is "0+unknown", but full hex is still there - if verbose: - print("no suitable tags, using unknown + full revision id") - return { - "version": "0+unknown", - "full-revisionid": keywords["full"].strip(), - "dirty": False, - "error": "no suitable tags", - "date": None, - } - - -@register_vcs_handler("git", "pieces_from_vcs") -def git_pieces_from_vcs(tag_prefix, root, verbose, run_command=run_command): - """Get version from 'git describe' in the root of the source tree. - - This only gets called if the git-archive 'subst' keywords were *not* - expanded, and _version.py hasn't already been rewritten with a short - version string, meaning we're inside a checked out source tree. - """ - GITS = ["git"] - if sys.platform == "win32": - GITS = ["git.cmd", "git.exe"] - - out, rc = run_command( - GITS, ["rev-parse", "--git-dir"], cwd=root, hide_stderr=True - ) - if rc != 0: - if verbose: - print("Directory %s not under git control" % root) - raise NotThisMethod("'git rev-parse --git-dir' returned error") - - # if there is a tag matching tag_prefix, this yields TAG-NUM-gHEX[-dirty] - # if there isn't one, this yields HEX[-dirty] (no NUM) - describe_out, rc = run_command( - GITS, - [ - "describe", - "--tags", - "--dirty", - "--always", - "--long", - "--match", - "%s*" % tag_prefix, - ], - cwd=root, - ) - # --long was added in git-1.5.5 - if describe_out is None: - raise NotThisMethod("'git describe' failed") - describe_out = describe_out.strip() - full_out, rc = run_command(GITS, ["rev-parse", "HEAD"], cwd=root) - if full_out is None: - raise NotThisMethod("'git rev-parse' failed") - full_out = full_out.strip() - - pieces = {} - pieces["long"] = full_out - pieces["short"] = full_out[:7] # maybe improved later - pieces["error"] = None - - # parse describe_out. It will be like TAG-NUM-gHEX[-dirty] or HEX[-dirty] - # TAG might have hyphens. - git_describe = describe_out - - # look for -dirty suffix - dirty = git_describe.endswith("-dirty") - pieces["dirty"] = dirty - if dirty: - git_describe = git_describe[: git_describe.rindex("-dirty")] - - # now we have TAG-NUM-gHEX or HEX - - if "-" in git_describe: - # TAG-NUM-gHEX - mo = re.search(r"^(.+)-(\d+)-g([0-9a-f]+)$", git_describe) - if not mo: - # unparseable. Maybe git-describe is misbehaving? - pieces["error"] = ( - "unable to parse git-describe output: '%s'" % describe_out - ) - return pieces - - # tag - full_tag = mo.group(1) - if not full_tag.startswith(tag_prefix): - if verbose: - fmt = "tag '%s' doesn't start with prefix '%s'" - print(fmt % (full_tag, tag_prefix)) - pieces["error"] = "tag '%s' doesn't start with prefix '%s'" % ( - full_tag, - tag_prefix, - ) - return pieces - pieces["closest-tag"] = full_tag[len(tag_prefix) :] - - # distance: number of commits since tag - pieces["distance"] = int(mo.group(2)) - - # commit: short hex revision ID - pieces["short"] = mo.group(3) - - else: - # HEX: no tags - pieces["closest-tag"] = None - count_out, rc = run_command( - GITS, ["rev-list", "HEAD", "--count"], cwd=root - ) - pieces["distance"] = int(count_out) # total number of commits - - # commit date: see ISO-8601 comment in git_versions_from_keywords() - date = run_command(GITS, ["show", "-s", "--format=%ci", "HEAD"], cwd=root)[ - 0 - ].strip() - pieces["date"] = date.strip().replace(" ", "T", 1).replace(" ", "", 1) - - return pieces - - -def do_vcs_install(manifest_in, versionfile_source, ipy): - """Git-specific installation logic for Versioneer. - - For Git, this means creating/changing .gitattributes to mark _version.py - for export-subst keyword substitution. - """ - GITS = ["git"] - if sys.platform == "win32": - GITS = ["git.cmd", "git.exe"] - files = [manifest_in, versionfile_source] - if ipy: - files.append(ipy) - try: - me = __file__ - if me.endswith(".pyc") or me.endswith(".pyo"): - me = os.path.splitext(me)[0] + ".py" - versioneer_file = os.path.relpath(me) - except NameError: - versioneer_file = "versioneer.py" - files.append(versioneer_file) - present = False - try: - f = open(".gitattributes", "r") - for line in f.readlines(): - if line.strip().startswith(versionfile_source): - if "export-subst" in line.strip().split()[1:]: - present = True - f.close() - except EnvironmentError: - pass - if not present: - f = open(".gitattributes", "a+") - f.write("%s export-subst\n" % versionfile_source) - f.close() - files.append(".gitattributes") - run_command(GITS, ["add", "--"] + files) - - -def versions_from_parentdir(parentdir_prefix, root, verbose): - """Try to determine the version from the parent directory name. - - Source tarballs conventionally unpack into a directory that includes both - the project name and a version string. We will also support searching up - two directory levels for an appropriately named parent directory - """ - rootdirs = [] - - for i in range(3): - dirname = os.path.basename(root) - if dirname.startswith(parentdir_prefix): - return { - "version": dirname[len(parentdir_prefix) :], - "full-revisionid": None, - "dirty": False, - "error": None, - "date": None, - } - else: - rootdirs.append(root) - root = os.path.dirname(root) # up a level - - if verbose: - print( - "Tried directories %s but none started with prefix %s" - % (str(rootdirs), parentdir_prefix) - ) - raise NotThisMethod("rootdir doesn't start with parentdir_prefix") - - -SHORT_VERSION_PY = """ -# This file was generated by 'versioneer.py' (0.18) from -# revision-control system data, or from the parent directory name of an -# unpacked source archive. Distribution tarballs contain a pre-generated copy -# of this file. - -import json - -version_json = ''' -%s -''' # END VERSION_JSON - - -def get_versions(): - return json.loads(version_json) -""" - - -def versions_from_file(filename): - """Try to determine the version from _version.py if present.""" - try: - with open(filename) as f: - contents = f.read() - except EnvironmentError: - raise NotThisMethod("unable to read _version.py") - mo = re.search( - r"version_json = '''\n(.*)''' # END VERSION_JSON", - contents, - re.M | re.S, - ) - if not mo: - mo = re.search( - r"version_json = '''\r\n(.*)''' # END VERSION_JSON", - contents, - re.M | re.S, - ) - if not mo: - raise NotThisMethod("no version_json in _version.py") - return json.loads(mo.group(1)) - - -def write_to_version_file(filename, versions): - """Write the given version number to the given _version.py file.""" - os.unlink(filename) - contents = json.dumps( - versions, sort_keys=True, indent=1, separators=(",", ": ") - ) - with open(filename, "w") as f: - f.write(SHORT_VERSION_PY % contents) - - print("set %s to '%s'" % (filename, versions["version"])) - - -def plus_or_dot(pieces): - """Return a + if we don't already have one, else return a .""" - if "+" in pieces.get("closest-tag", ""): - return "." - return "+" - - -def render_pep440(pieces): - """Build up version string, with post-release "local version identifier". - - Our goal: TAG[+DISTANCE.gHEX[.dirty]] . Note that if you - get a tagged build and then dirty it, you'll get TAG+0.gHEX.dirty - - Exceptions: - 1: no tags. git_describe was just HEX. 0+untagged.DISTANCE.gHEX[.dirty] - """ - if pieces["closest-tag"]: - rendered = pieces["closest-tag"] - if pieces["distance"] or pieces["dirty"]: - rendered += plus_or_dot(pieces) - rendered += "%d.g%s" % (pieces["distance"], pieces["short"]) - if pieces["dirty"]: - rendered += ".dirty" - else: - # exception #1 - rendered = "0+untagged.%d.g%s" % (pieces["distance"], pieces["short"]) - if pieces["dirty"]: - rendered += ".dirty" - return rendered - - -def render_pep440_pre(pieces): - """TAG[.post.devDISTANCE] -- No -dirty. - - Exceptions: - 1: no tags. 0.post.devDISTANCE - """ - if pieces["closest-tag"]: - rendered = pieces["closest-tag"] - if pieces["distance"]: - rendered += ".post.dev%d" % pieces["distance"] - else: - # exception #1 - rendered = "0.post.dev%d" % pieces["distance"] - return rendered - - -def render_pep440_post(pieces): - """TAG[.postDISTANCE[.dev0]+gHEX] . - - The ".dev0" means dirty. Note that .dev0 sorts backwards - (a dirty tree will appear "older" than the corresponding clean one), - but you shouldn't be releasing software with -dirty anyways. - - Exceptions: - 1: no tags. 0.postDISTANCE[.dev0] - """ - if pieces["closest-tag"]: - rendered = pieces["closest-tag"] - if pieces["distance"] or pieces["dirty"]: - rendered += ".post%d" % pieces["distance"] - if pieces["dirty"]: - rendered += ".dev0" - rendered += plus_or_dot(pieces) - rendered += "g%s" % pieces["short"] - else: - # exception #1 - rendered = "0.post%d" % pieces["distance"] - if pieces["dirty"]: - rendered += ".dev0" - rendered += "+g%s" % pieces["short"] - return rendered - - -def render_pep440_old(pieces): - """TAG[.postDISTANCE[.dev0]] . - - The ".dev0" means dirty. - - Exceptions: - 1: no tags. 0.postDISTANCE[.dev0] - """ - if pieces["closest-tag"]: - rendered = pieces["closest-tag"] - if pieces["distance"] or pieces["dirty"]: - rendered += ".post%d" % pieces["distance"] - if pieces["dirty"]: - rendered += ".dev0" - else: - # exception #1 - rendered = "0.post%d" % pieces["distance"] - if pieces["dirty"]: - rendered += ".dev0" - return rendered - - -def render_git_describe(pieces): - """TAG[-DISTANCE-gHEX][-dirty]. - - Like 'git describe --tags --dirty --always'. - - Exceptions: - 1: no tags. HEX[-dirty] (note: no 'g' prefix) - """ - if pieces["closest-tag"]: - rendered = pieces["closest-tag"] - if pieces["distance"]: - rendered += "-%d-g%s" % (pieces["distance"], pieces["short"]) - else: - # exception #1 - rendered = pieces["short"] - if pieces["dirty"]: - rendered += "-dirty" - return rendered - - -def render_git_describe_long(pieces): - """TAG-DISTANCE-gHEX[-dirty]. - - Like 'git describe --tags --dirty --always -long'. - The distance/hash is unconditional. - - Exceptions: - 1: no tags. HEX[-dirty] (note: no 'g' prefix) - """ - if pieces["closest-tag"]: - rendered = pieces["closest-tag"] - rendered += "-%d-g%s" % (pieces["distance"], pieces["short"]) - else: - # exception #1 - rendered = pieces["short"] - if pieces["dirty"]: - rendered += "-dirty" - return rendered - - -def render(pieces, style): - """Render the given version pieces into the requested style.""" - if pieces["error"]: - return { - "version": "unknown", - "full-revisionid": pieces.get("long"), - "dirty": None, - "error": pieces["error"], - "date": None, - } - - if not style or style == "default": - style = "pep440" # the default - - if style == "pep440": - rendered = render_pep440(pieces) - elif style == "pep440-pre": - rendered = render_pep440_pre(pieces) - elif style == "pep440-post": - rendered = render_pep440_post(pieces) - elif style == "pep440-old": - rendered = render_pep440_old(pieces) - elif style == "git-describe": - rendered = render_git_describe(pieces) - elif style == "git-describe-long": - rendered = render_git_describe_long(pieces) - else: - raise ValueError("unknown style '%s'" % style) - - return { - "version": rendered, - "full-revisionid": pieces["long"], - "dirty": pieces["dirty"], - "error": None, - "date": pieces.get("date"), - } - - -class VersioneerBadRootError(Exception): - """The project root directory is unknown or missing key files.""" - - -def get_versions(verbose=False): - """Get the project version from whatever source is available. - - Returns dict with two keys: 'version' and 'full'. - """ - if "versioneer" in sys.modules: - # see the discussion in cmdclass.py:get_cmdclass() - del sys.modules["versioneer"] - - root = get_root() - cfg = get_config_from_root(root) - - assert cfg.VCS is not None, "please set [versioneer]VCS= in setup.cfg" - handlers = HANDLERS.get(cfg.VCS) - assert handlers, "unrecognized VCS '%s'" % cfg.VCS - verbose = verbose or cfg.verbose - assert ( - cfg.versionfile_source is not None - ), "please set versioneer.versionfile_source" - assert cfg.tag_prefix is not None, "please set versioneer.tag_prefix" - - versionfile_abs = os.path.join(root, cfg.versionfile_source) - - # extract version from first of: _version.py, VCS command (e.g. 'git - # describe'), parentdir. This is meant to work for developers using a - # source checkout, for users of a tarball created by 'setup.py sdist', - # and for users of a tarball/zipball created by 'git archive' or github's - # download-from-tag feature or the equivalent in other VCSes. - - get_keywords_f = handlers.get("get_keywords") - from_keywords_f = handlers.get("keywords") - if get_keywords_f and from_keywords_f: - try: - keywords = get_keywords_f(versionfile_abs) - ver = from_keywords_f(keywords, cfg.tag_prefix, verbose) - if verbose: - print("got version from expanded keyword %s" % ver) - return ver - except NotThisMethod: - pass - - try: - ver = versions_from_file(versionfile_abs) - if verbose: - print("got version from file %s %s" % (versionfile_abs, ver)) - return ver - except NotThisMethod: - pass - - from_vcs_f = handlers.get("pieces_from_vcs") - if from_vcs_f: - try: - pieces = from_vcs_f(cfg.tag_prefix, root, verbose) - ver = render(pieces, cfg.style) - if verbose: - print("got version from VCS %s" % ver) - return ver - except NotThisMethod: - pass - - try: - if cfg.parentdir_prefix: - ver = versions_from_parentdir(cfg.parentdir_prefix, root, verbose) - if verbose: - print("got version from parentdir %s" % ver) - return ver - except NotThisMethod: - pass - - if verbose: - print("unable to compute version") - - return { - "version": "0+unknown", - "full-revisionid": None, - "dirty": None, - "error": "unable to compute version", - "date": None, - } - - -def get_version(): - """Get the short version string for this project.""" - return get_versions()["version"] - - -def get_cmdclass(): - """Get the custom setuptools/distutils subclasses used by Versioneer.""" - if "versioneer" in sys.modules: - del sys.modules["versioneer"] - # this fixes the "python setup.py develop" case (also 'install' and - # 'easy_install .'), in which subdependencies of the main project are - # built (using setup.py bdist_egg) in the same python process. Assume - # a main project A and a dependency B, which use different versions - # of Versioneer. A's setup.py imports A's Versioneer, leaving it in - # sys.modules by the time B's setup.py is executed, causing B to run - # with the wrong versioneer. Setuptools wraps the sub-dep builds in a - # sandbox that restores sys.modules to it's pre-build state, so the - # parent is protected against the child's "import versioneer". By - # removing ourselves from sys.modules here, before the child build - # happens, we protect the child from the parent's versioneer too. - # Also see https://github.com/warner/python-versioneer/issues/52 - - cmds = {} - - # we add "version" to both distutils and setuptools - from distutils.core import Command - - class cmd_version(Command): - description = "report generated version string" - user_options = [] - boolean_options = [] - - def initialize_options(self): - pass - - def finalize_options(self): - pass - - def run(self): - vers = get_versions(verbose=True) - print("Version: %s" % vers["version"]) - print(" full-revisionid: %s" % vers.get("full-revisionid")) - print(" dirty: %s" % vers.get("dirty")) - print(" date: %s" % vers.get("date")) - if vers["error"]: - print(" error: %s" % vers["error"]) - - cmds["version"] = cmd_version - - # we override "build_py" in both distutils and setuptools - # - # most invocation pathways end up running build_py: - # distutils/build -> build_py - # distutils/install -> distutils/build ->.. - # setuptools/bdist_wheel -> distutils/install ->.. - # setuptools/bdist_egg -> distutils/install_lib -> build_py - # setuptools/install -> bdist_egg ->.. - # setuptools/develop -> ? - # pip install: - # copies source tree to a tempdir before running egg_info/etc - # if .git isn't copied too, 'git describe' will fail - # then does setup.py bdist_wheel, or sometimes setup.py install - # setup.py egg_info -> ? - - # we override different "build_py" commands for both environments - if "setuptools" in sys.modules: - from setuptools.command.build_py import build_py as _build_py - else: - from distutils.command.build_py import build_py as _build_py - - class cmd_build_py(_build_py): - def run(self): - root = get_root() - cfg = get_config_from_root(root) - versions = get_versions() - _build_py.run(self) - # now locate _version.py in the new build/ directory and replace - # it with an updated value - if cfg.versionfile_build: - target_versionfile = os.path.join( - self.build_lib, cfg.versionfile_build - ) - print("UPDATING %s" % target_versionfile) - write_to_version_file(target_versionfile, versions) - - cmds["build_py"] = cmd_build_py - - if "cx_Freeze" in sys.modules: # cx_freeze enabled? - from cx_Freeze.dist import build_exe as _build_exe - - # nczeczulin reports that py2exe won't like the pep440-style string - # as FILEVERSION, but it can be used for PRODUCTVERSION, e.g. - # setup(console=[{ - # "version": versioneer.get_version().split("+", 1)[0], # FILEVERSION - # "product_version": versioneer.get_version(), - # ... - - class cmd_build_exe(_build_exe): - def run(self): - root = get_root() - cfg = get_config_from_root(root) - versions = get_versions() - target_versionfile = cfg.versionfile_source - print("UPDATING %s" % target_versionfile) - write_to_version_file(target_versionfile, versions) - - _build_exe.run(self) - os.unlink(target_versionfile) - with open(cfg.versionfile_source, "w") as f: - LONG = LONG_VERSION_PY[cfg.VCS] - f.write( - LONG - % { - "DOLLAR": "$", - "STYLE": cfg.style, - "TAG_PREFIX": cfg.tag_prefix, - "PARENTDIR_PREFIX": cfg.parentdir_prefix, - "VERSIONFILE_SOURCE": cfg.versionfile_source, - } - ) - - cmds["build_exe"] = cmd_build_exe - del cmds["build_py"] - - if "py2exe" in sys.modules: # py2exe enabled? - try: - from py2exe.distutils_buildexe import py2exe as _py2exe # py3 - except ImportError: - from py2exe.build_exe import py2exe as _py2exe # py2 - - class cmd_py2exe(_py2exe): - def run(self): - root = get_root() - cfg = get_config_from_root(root) - versions = get_versions() - target_versionfile = cfg.versionfile_source - print("UPDATING %s" % target_versionfile) - write_to_version_file(target_versionfile, versions) - - _py2exe.run(self) - os.unlink(target_versionfile) - with open(cfg.versionfile_source, "w") as f: - LONG = LONG_VERSION_PY[cfg.VCS] - f.write( - LONG - % { - "DOLLAR": "$", - "STYLE": cfg.style, - "TAG_PREFIX": cfg.tag_prefix, - "PARENTDIR_PREFIX": cfg.parentdir_prefix, - "VERSIONFILE_SOURCE": cfg.versionfile_source, - } - ) - - cmds["py2exe"] = cmd_py2exe - - # we override different "sdist" commands for both environments - if "setuptools" in sys.modules: - from setuptools.command.sdist import sdist as _sdist - else: - from distutils.command.sdist import sdist as _sdist - - class cmd_sdist(_sdist): - def run(self): - versions = get_versions() - self._versioneer_generated_versions = versions - # unless we update this, the command will keep using the old - # version - self.distribution.metadata.version = versions["version"] - return _sdist.run(self) - - def make_release_tree(self, base_dir, files): - root = get_root() - cfg = get_config_from_root(root) - _sdist.make_release_tree(self, base_dir, files) - # now locate _version.py in the new base_dir directory - # (remembering that it may be a hardlink) and replace it with an - # updated value - target_versionfile = os.path.join(base_dir, cfg.versionfile_source) - print("UPDATING %s" % target_versionfile) - write_to_version_file( - target_versionfile, self._versioneer_generated_versions - ) - - cmds["sdist"] = cmd_sdist - - return cmds - - -CONFIG_ERROR = """ -setup.cfg is missing the necessary Versioneer configuration. You need -a section like: - - [versioneer] - VCS = git - style = pep440 - versionfile_source = src/myproject/_version.py - versionfile_build = myproject/_version.py - tag_prefix = - parentdir_prefix = myproject- - -You will also need to edit your setup.py to use the results: - - import versioneer - setup(version=versioneer.get_version(), - cmdclass=versioneer.get_cmdclass(), ...) - -Please read the docstring in ./versioneer.py for configuration instructions, -edit setup.cfg, and re-run the installer or 'python versioneer.py setup'. -""" - -SAMPLE_CONFIG = """ -# See the docstring in versioneer.py for instructions. Note that you must -# re-run 'versioneer.py setup' after changing this section, and commit the -# resulting files. - -[versioneer] -#VCS = git -#style = pep440 -#versionfile_source = -#versionfile_build = -#tag_prefix = -#parentdir_prefix = - -""" - -INIT_PY_SNIPPET = """ -from ._version import get_versions -__version__ = get_versions()['version'] -del get_versions -""" - - -def do_setup(): - """Main VCS-independent setup function for installing Versioneer.""" - root = get_root() - try: - cfg = get_config_from_root(root) - except ( - EnvironmentError, - configparser.NoSectionError, - configparser.NoOptionError, - ) as e: - if isinstance(e, (EnvironmentError, configparser.NoSectionError)): - print( - "Adding sample versioneer config to setup.cfg", file=sys.stderr - ) - with open(os.path.join(root, "setup.cfg"), "a") as f: - f.write(SAMPLE_CONFIG) - print(CONFIG_ERROR, file=sys.stderr) - return 1 - - print(" creating %s" % cfg.versionfile_source) - with open(cfg.versionfile_source, "w") as f: - LONG = LONG_VERSION_PY[cfg.VCS] - f.write( - LONG - % { - "DOLLAR": "$", - "STYLE": cfg.style, - "TAG_PREFIX": cfg.tag_prefix, - "PARENTDIR_PREFIX": cfg.parentdir_prefix, - "VERSIONFILE_SOURCE": cfg.versionfile_source, - } - ) - - ipy = os.path.join(os.path.dirname(cfg.versionfile_source), "__init__.py") - if os.path.exists(ipy): - try: - with open(ipy, "r") as f: - old = f.read() - except EnvironmentError: - old = "" - if INIT_PY_SNIPPET not in old: - print(" appending to %s" % ipy) - with open(ipy, "a") as f: - f.write(INIT_PY_SNIPPET) - else: - print(" %s unmodified" % ipy) - else: - print(" %s doesn't exist, ok" % ipy) - ipy = None - - # Make sure both the top-level "versioneer.py" and versionfile_source - # (PKG/_version.py, used by runtime code) are in MANIFEST.in, so - # they'll be copied into source distributions. Pip won't be able to - # install the package without this. - manifest_in = os.path.join(root, "MANIFEST.in") - simple_includes = set() - try: - with open(manifest_in, "r") as f: - for line in f: - if line.startswith("include "): - for include in line.split()[1:]: - simple_includes.add(include) - except EnvironmentError: - pass - # That doesn't cover everything MANIFEST.in can do - # (http://docs.python.org/2/distutils/sourcedist.html#commands), so - # it might give some false negatives. Appending redundant 'include' - # lines is safe, though. - if "versioneer.py" not in simple_includes: - print(" appending 'versioneer.py' to MANIFEST.in") - with open(manifest_in, "a") as f: - f.write("include versioneer.py\n") - else: - print(" 'versioneer.py' already in MANIFEST.in") - if cfg.versionfile_source not in simple_includes: - print( - " appending versionfile_source ('%s') to MANIFEST.in" - % cfg.versionfile_source - ) - with open(manifest_in, "a") as f: - f.write("include %s\n" % cfg.versionfile_source) - else: - print(" versionfile_source already in MANIFEST.in") - - # Make VCS-specific changes. For git, this means creating/changing - # .gitattributes to mark _version.py for export-subst keyword - # substitution. - do_vcs_install(manifest_in, cfg.versionfile_source, ipy) - return 0 - - -def scan_setup_py(): - """Validate the contents of setup.py against Versioneer's expectations.""" - found = set() - setters = False - errors = 0 - with open("setup.py", "r") as f: - for line in f.readlines(): - if "import versioneer" in line: - found.add("import") - if "versioneer.get_cmdclass()" in line: - found.add("cmdclass") - if "versioneer.get_version()" in line: - found.add("get_version") - if "versioneer.VCS" in line: - setters = True - if "versioneer.versionfile_source" in line: - setters = True - if len(found) != 3: - print("") - print("Your setup.py appears to be missing some important items") - print("(but I might be wrong). Please make sure it has something") - print("roughly like the following:") - print("") - print(" import versioneer") - print(" setup( version=versioneer.get_version(),") - print(" cmdclass=versioneer.get_cmdclass(), ...)") - print("") - errors += 1 - if setters: - print("You should remove lines like 'versioneer.VCS = ' and") - print("'versioneer.versionfile_source = ' . This configuration") - print("now lives in setup.cfg, and should be removed from setup.py") - print("") - errors += 1 - return errors - - -if __name__ == "__main__": - cmd = sys.argv[1] - if cmd == "setup": - errors = do_setup() - errors += scan_setup_py() - if errors: - sys.exit(1) diff --git a/setup.cfg b/setup.cfg index 3f8fa7e8406..962b7d73bbe 100644 --- a/setup.cfg +++ b/setup.cfg @@ -1,4 +1,4 @@ -# Copyright (c) 2017-2022, NVIDIA CORPORATION. +# Copyright (c) 2017-2023, NVIDIA CORPORATION. [flake8] filename = *.py, *.pyx, *.pxd, *.pxi @@ -44,8 +44,7 @@ ignore_missing_imports = True # they are imported by a checked file. follow_imports = skip exclude = (?x)( - (cudf|custreamz|cudf_kafka|dask_cudf)/_version\.py - | cudf/_lib/ + cudf/_lib/ | cudf/cudf/benchmarks/ | cudf/cudf/tests/ | cudf/cudf/utils/metadata/orc_column_statistics_pb2.py @@ -57,7 +56,7 @@ exclude = (?x)( [codespell] # note: pre-commit passes explicit lists of files here, which this skip file list doesn't override - # this is only to allow you to run codespell interactively -skip = ./.git,./.github,./cpp/build,.*egg-info.*,versioneer.py,./.mypy_cache,./cpp/tests,./python/cudf/cudf/tests,./java/src/test,./cpp/include/cudf_test/cxxopts.hpp +skip = ./.git,./.github,./cpp/build,.*egg-info.*,./.mypy_cache,./cpp/tests,./python/cudf/cudf/tests,./java/src/test,./cpp/include/cudf_test/cxxopts.hpp # ignore short words, and typename parameters like OffsetT ignore-regex = \b(.{1,4}|[A-Z]\w*T)\b ignore-words-list = inout,unparseable From f076905f53f3f2d44ada244c0754afc459b306a1 Mon Sep 17 00:00:00 2001 From: David Wendt <45795991+davidwendt@users.noreply.github.com> Date: Wed, 22 Feb 2023 20:52:02 -0500 Subject: [PATCH 04/60] Add segmented reduction support for fixed-point types (#12680) Depends on #12573 Adds additional support for fixed-point types in `cudf::segmented_reduce` for simple aggregations: sum, product, and sum-of-squares. Reference: #10432 Authors: - David Wendt (https://github.com/davidwendt) Approvers: - Mike Wilson (https://github.com/hyperbolic2346) - Nghia Truong (https://github.com/ttnghia) - Bradley Dice (https://github.com/bdice) URL: https://github.com/rapidsai/cudf/pull/12680 --- cpp/CMakeLists.txt | 1 + cpp/src/reductions/segmented/compound.cuh | 29 +- cpp/src/reductions/segmented/counts.cu | 54 ++++ cpp/src/reductions/segmented/counts.hpp | 55 ++++ cpp/src/reductions/segmented/simple.cuh | 91 ++++-- .../reductions/segmented_reduction_tests.cpp | 305 +++++++++--------- 6 files changed, 341 insertions(+), 194 deletions(-) create mode 100644 cpp/src/reductions/segmented/counts.cu create mode 100644 cpp/src/reductions/segmented/counts.hpp diff --git a/cpp/CMakeLists.txt b/cpp/CMakeLists.txt index d402a47628c..96524b7c55f 100644 --- a/cpp/CMakeLists.txt +++ b/cpp/CMakeLists.txt @@ -455,6 +455,7 @@ add_library( src/reductions/scan/scan_inclusive.cu src/reductions/segmented/all.cu src/reductions/segmented/any.cu + src/reductions/segmented/counts.cu src/reductions/segmented/max.cu src/reductions/segmented/mean.cu src/reductions/segmented/min.cu diff --git a/cpp/src/reductions/segmented/compound.cuh b/cpp/src/reductions/segmented/compound.cuh index dc8a995d1b0..e8abd32cf61 100644 --- a/cpp/src/reductions/segmented/compound.cuh +++ b/cpp/src/reductions/segmented/compound.cuh @@ -16,6 +16,7 @@ #pragma once +#include "counts.hpp" #include "update_validity.hpp" #include @@ -63,34 +64,26 @@ std::unique_ptr compound_segmented_reduction(column_view const& col, data_type{type_to_id()}, num_segments, mask_state::UNALLOCATED, stream, mr); auto out_itr = result->mutable_view().template begin(); - // Compute valid counts - rmm::device_uvector valid_counts(num_segments, stream); - if (col.has_nulls() && (null_handling == null_policy::EXCLUDE)) { - auto valid_fn = [] __device__(auto p) -> size_type { return static_cast(p.second); }; - auto itr = thrust::make_transform_iterator(d_col->pair_begin(), valid_fn); - cudf::reduction::detail::segmented_reduce(itr, - offsets.begin(), - offsets.end(), - valid_counts.data(), - thrust::plus{}, - 0, - stream); - } else { - thrust::adjacent_difference( - rmm::exec_policy(stream), offsets.begin() + 1, offsets.end(), valid_counts.begin()); - } + // Compute counts + rmm::device_uvector counts = + cudf::reduction::detail::segmented_counts(col.null_mask(), + col.has_nulls(), + offsets, + null_handling, + stream, + rmm::mr::get_current_device_resource()); // Run segmented reduction if (col.has_nulls()) { auto nrt = compound_op.template get_null_replacing_element_transformer(); auto itr = thrust::make_transform_iterator(d_col->pair_begin(), nrt); cudf::reduction::detail::segmented_reduce( - itr, offsets.begin(), offsets.end(), out_itr, compound_op, ddof, valid_counts.data(), stream); + itr, offsets.begin(), offsets.end(), out_itr, compound_op, ddof, counts.data(), stream); } else { auto et = compound_op.template get_element_transformer(); auto itr = thrust::make_transform_iterator(d_col->begin(), et); cudf::reduction::detail::segmented_reduce( - itr, offsets.begin(), offsets.end(), out_itr, compound_op, ddof, valid_counts.data(), stream); + itr, offsets.begin(), offsets.end(), out_itr, compound_op, ddof, counts.data(), stream); } // Compute the output null mask diff --git a/cpp/src/reductions/segmented/counts.cu b/cpp/src/reductions/segmented/counts.cu new file mode 100644 index 00000000000..b9064ad3ffe --- /dev/null +++ b/cpp/src/reductions/segmented/counts.cu @@ -0,0 +1,54 @@ +/* + * Copyright (c) 2023, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include "counts.hpp" + +#include + +#include + +namespace cudf { +namespace reduction { +namespace detail { + +rmm::device_uvector segmented_counts(bitmask_type const* null_mask, + bool has_nulls, + device_span offsets, + null_policy null_handling, + rmm::cuda_stream_view stream, + rmm::mr::device_memory_resource* mr) +{ + auto const num_segments = offsets.size() - 1; + + if (has_nulls && (null_handling == null_policy::EXCLUDE)) { + return cudf::detail::segmented_count_bits(null_mask, + offsets.begin(), + offsets.end() - 1, + offsets.begin() + 1, + cudf::detail::count_bits_policy::SET_BITS, + stream, + mr); + } + + rmm::device_uvector valid_counts(num_segments, stream, mr); + thrust::adjacent_difference( + rmm::exec_policy(stream), offsets.begin() + 1, offsets.end(), valid_counts.begin()); + return valid_counts; +} + +} // namespace detail +} // namespace reduction +} // namespace cudf diff --git a/cpp/src/reductions/segmented/counts.hpp b/cpp/src/reductions/segmented/counts.hpp new file mode 100644 index 00000000000..c5ee1fadae7 --- /dev/null +++ b/cpp/src/reductions/segmented/counts.hpp @@ -0,0 +1,55 @@ +/* + * Copyright (c) 2023, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#pragma once + +#include +#include + +#include +#include + +namespace cudf { +class column_device_view; + +namespace reduction { +namespace detail { + +/** + * @brief Compute the number of elements per segment + * + * If `null_handling == null_policy::EXCLUDE`, the count for each + * segment omits any null entries. Otherwise, this returns the number + * of elements in each segment. + * + * @param null_mask Null values over which the segment offsets apply + * @param has_nulls True if d_col contains any nulls + * @param offsets Indices to segment boundaries + * @param null_handling How null entries are processed within each segment + * @param stream Used for device memory operations and kernel launches + * @param mr Device memory resource used to allocate the returned column's device memory + * @return The number of elements in each segment + */ +rmm::device_uvector segmented_counts(bitmask_type const* null_mask, + bool has_nulls, + device_span offsets, + null_policy null_handling, + rmm::cuda_stream_view stream, + rmm::mr::device_memory_resource* mr); + +} // namespace detail +} // namespace reduction +} // namespace cudf diff --git a/cpp/src/reductions/segmented/simple.cuh b/cpp/src/reductions/segmented/simple.cuh index fb080ebf67c..0c22848fd89 100644 --- a/cpp/src/reductions/segmented/simple.cuh +++ b/cpp/src/reductions/segmented/simple.cuh @@ -16,6 +16,7 @@ #pragma once +#include "counts.hpp" #include "update_validity.hpp" #include @@ -36,6 +37,7 @@ #include #include #include +#include #include #include @@ -188,7 +190,7 @@ std::unique_ptr string_segmented_reduction(column_view const& col, } /** - * @brief Fixed point segmented reduction for 'min', 'max'. + * @brief Specialization for fixed-point segmented reduction * * @tparam InputType the input column data-type * @tparam Op the operator of cudf::reduction::op:: @@ -200,11 +202,7 @@ std::unique_ptr string_segmented_reduction(column_view const& col, * @param mr Device memory resource used to allocate the returned column's device memory * @return Output column in device memory */ - -template || - std::is_same_v)> +template std::unique_ptr fixed_point_segmented_reduction( column_view const& col, device_span offsets, @@ -214,23 +212,55 @@ std::unique_ptr fixed_point_segmented_reduction( rmm::mr::device_memory_resource* mr) { using RepType = device_storage_type_t; - return simple_segmented_reduction( - col, offsets, null_handling, init, stream, mr); -} + auto result = + simple_segmented_reduction(col, offsets, null_handling, init, stream, mr); + auto const scale = [&] { + if constexpr (std::is_same_v) { + // The product aggregation requires updating the scale of the fixed-point output column. + // The output scale needs to be the maximum count of all segments multiplied by + // the input scale value. + rmm::device_uvector const counts = + cudf::reduction::detail::segmented_counts(col.null_mask(), + col.has_nulls(), + offsets, + null_policy::EXCLUDE, // do not count nulls + stream, + rmm::mr::get_current_device_resource()); + + auto const max_count = thrust::reduce(rmm::exec_policy(stream), + counts.begin(), + counts.end(), + size_type{0}, + thrust::maximum{}); + + auto const new_scale = numeric::scale_type{col.type().scale() * max_count}; + + // adjust values in each segment to match the new scale + auto const d_col = column_device_view::create(col, stream); + thrust::transform(rmm::exec_policy(stream), + d_col->begin(), + d_col->end(), + d_col->begin(), + [new_scale] __device__(auto fp) { return fp.rescaled(new_scale); }); + return new_scale; + } -template () && - !std::is_same_v())> -std::unique_ptr fixed_point_segmented_reduction( - column_view const& col, - device_span offsets, - null_policy null_handling, - std::optional>, - rmm::cuda_stream_view stream, - rmm::mr::device_memory_resource* mr) -{ - CUDF_FAIL("Segmented reduction on fixed point column only supports min and max reduction."); + if constexpr (std::is_same_v) { + return numeric::scale_type{col.type().scale() * 2}; + } + + return numeric::scale_type{col.type().scale()}; + }(); + + auto const size = result->size(); // get these before + auto const null_count = result->null_count(); // release() is called + auto contents = result->release(); + + return std::make_unique(data_type{type_to_id(), scale}, + size, + std::move(*(contents.data.release())), + std::move(*(contents.null_mask.release())), + null_count); } /** @@ -431,8 +461,23 @@ struct column_type_dispatcher { return reduce_numeric(col, offsets, output_type, null_handling, init, stream, mr); } + template ()>* = nullptr> + std::unique_ptr operator()(column_view const& col, + device_span offsets, + data_type const output_type, + null_policy null_handling, + std::optional> init, + rmm::cuda_stream_view stream, + rmm::mr::device_memory_resource* mr) + { + CUDF_EXPECTS(output_type == col.type(), "Output type must be same as input column type."); + return fixed_point_segmented_reduction( + col, offsets, null_handling, init, stream, mr); + } + template ()>* = nullptr> + std::enable_if_t() and + not cudf::is_fixed_point()>* = nullptr> std::unique_ptr operator()(column_view const&, device_span, data_type const, diff --git a/cpp/tests/reductions/segmented_reduction_tests.cpp b/cpp/tests/reductions/segmented_reduction_tests.cpp index b4873a14509..74c5e7fb504 100644 --- a/cpp/tests/reductions/segmented_reduction_tests.cpp +++ b/cpp/tests/reductions/segmented_reduction_tests.cpp @@ -1094,223 +1094,222 @@ struct SegmentedReductionFixedPointTest : public cudf::test::BaseFixture { TYPED_TEST_SUITE(SegmentedReductionFixedPointTest, cudf::test::FixedPointTypes); -TYPED_TEST(SegmentedReductionFixedPointTest, MaxIncludeNulls) +TYPED_TEST(SegmentedReductionFixedPointTest, MaxWithNulls) { - // scale: -2, 0, 5 - // [1, 2, 3], [1, null, 3], [1], [null], [null, null], [] - // values: {1, 2, 3, 1, XXX, 3, 1, XXX, XXX, XXX} - // offsets: {0, 3, 6, 7, 8, 10, 10} - // nullmask: {1, 1, 1, 1, 0, 1, 1, 0, 0, 0} - // outputs: {3, XXX, 1, XXX, XXX, XXX} - // output nullmask: {1, 0, 1, 0, 0, 0} - using RepType = cudf::device_storage_type_t; + auto const offsets = std::vector{0, 3, 6, 7, 8, 10, 10}; + auto const d_offsets = + cudf::detail::make_device_uvector_async(offsets, cudf::get_default_stream()); + auto const agg = cudf::make_max_aggregation(); + for (auto scale : {-2, 0, 5}) { auto const input = cudf::test::fixed_point_column_wrapper({1, 2, 3, 1, XXX, 3, 1, XXX, XXX, XXX}, {1, 1, 1, 1, 0, 1, 1, 0, 0, 0}, numeric::scale_type{scale}); - auto const offsets = std::vector{0, 3, 6, 7, 8, 10, 10}; - auto const d_offsets = - cudf::detail::make_device_uvector_async(offsets, cudf::get_default_stream()); - auto out_type = cudf::column_view(input).type(); - auto const expect = cudf::test::fixed_point_column_wrapper( + auto out_type = cudf::column_view(input).type(); + auto expect = cudf::test::fixed_point_column_wrapper( {3, XXX, 1, XXX, XXX, XXX}, {1, 0, 1, 0, 0, 0}, numeric::scale_type{scale}); + auto result = + cudf::segmented_reduce(input, d_offsets, *agg, out_type, cudf::null_policy::INCLUDE); + CUDF_TEST_EXPECT_COLUMNS_EQUAL(*result, expect); - auto res = - cudf::segmented_reduce(input, - d_offsets, - *cudf::make_max_aggregation(), - out_type, - cudf::null_policy::INCLUDE); - - CUDF_TEST_EXPECT_COLUMNS_EQUAL(*res, expect); + expect = cudf::test::fixed_point_column_wrapper( + {3, 3, 1, XXX, XXX, XXX}, {1, 1, 1, 0, 0, 0}, numeric::scale_type{scale}); + result = cudf::segmented_reduce(input, d_offsets, *agg, out_type, cudf::null_policy::EXCLUDE); + CUDF_TEST_EXPECT_COLUMNS_EQUAL(*result, expect); } } -TYPED_TEST(SegmentedReductionFixedPointTest, MaxExcludeNulls) +TYPED_TEST(SegmentedReductionFixedPointTest, MinWithNulls) { - // scale: -2, 0, 5 - // [1, 2, 3], [1, null, 3], [1], [null], [null, null], [] - // values: {1, 2, 3, 1, XXX, 3, 1, XXX, XXX, XXX} - // offsets: {0, 3, 6, 7, 8, 10, 10} - // nullmask: {1, 1, 1, 1, 0, 1, 1, 0, 0, 0} - // outputs: {3, 3, 1, XXX, XXX, XXX} - // output nullmask: {1, 1, 1, 0, 0, 0} - using RepType = cudf::device_storage_type_t; + auto const offsets = std::vector{0, 3, 6, 7, 8, 10, 10}; + auto const d_offsets = + cudf::detail::make_device_uvector_async(offsets, cudf::get_default_stream()); + auto const agg = cudf::make_min_aggregation(); + for (auto scale : {-2, 0, 5}) { auto const input = cudf::test::fixed_point_column_wrapper({1, 2, 3, 1, XXX, 3, 1, XXX, XXX, XXX}, {1, 1, 1, 1, 0, 1, 1, 0, 0, 0}, numeric::scale_type{scale}); - auto const offsets = std::vector{0, 3, 6, 7, 8, 10, 10}; - auto const d_offsets = - cudf::detail::make_device_uvector_async(offsets, cudf::get_default_stream()); + auto out_type = cudf::column_view(input).type(); + auto expect = cudf::test::fixed_point_column_wrapper( + {1, XXX, 1, XXX, XXX, XXX}, {1, 0, 1, 0, 0, 0}, numeric::scale_type{scale}); + auto result = + cudf::segmented_reduce(input, d_offsets, *agg, out_type, cudf::null_policy::INCLUDE); + CUDF_TEST_EXPECT_COLUMNS_EQUAL(*result, expect); + + expect = cudf::test::fixed_point_column_wrapper( + {1, 1, 1, XXX, XXX, XXX}, {1, 1, 1, 0, 0, 0}, numeric::scale_type{scale}); + result = cudf::segmented_reduce(input, d_offsets, *agg, out_type, cudf::null_policy::EXCLUDE); + CUDF_TEST_EXPECT_COLUMNS_EQUAL(*result, expect); + } +} + +TYPED_TEST(SegmentedReductionFixedPointTest, MaxNonNullableInput) +{ + using RepType = cudf::device_storage_type_t; + + auto const offsets = std::vector{0, 3, 4, 4}; + auto const d_offsets = + cudf::detail::make_device_uvector_async(offsets, cudf::get_default_stream()); + auto const agg = cudf::make_max_aggregation(); + + for (auto scale : {-2, 0, 5}) { + auto const input = + cudf::test::fixed_point_column_wrapper({1, 2, 3, 1}, numeric::scale_type{scale}); auto out_type = cudf::column_view(input).type(); auto const expect = cudf::test::fixed_point_column_wrapper( - {3, 3, 1, XXX, XXX, XXX}, {1, 1, 1, 0, 0, 0}, numeric::scale_type{scale}); + {3, 1, XXX}, {1, 1, 0}, numeric::scale_type{scale}); - auto res = - cudf::segmented_reduce(input, - d_offsets, - *cudf::make_max_aggregation(), - out_type, - cudf::null_policy::EXCLUDE); + auto result = + cudf::segmented_reduce(input, d_offsets, *agg, out_type, cudf::null_policy::INCLUDE); + CUDF_TEST_EXPECT_COLUMNS_EQUAL(*result, expect); - CUDF_TEST_EXPECT_COLUMNS_EQUAL(*res, expect); + result = cudf::segmented_reduce(input, d_offsets, *agg, out_type, cudf::null_policy::EXCLUDE); + CUDF_TEST_EXPECT_COLUMNS_EQUAL(*result, expect); } } -TYPED_TEST(SegmentedReductionFixedPointTest, MinIncludeNulls) +TYPED_TEST(SegmentedReductionFixedPointTest, MinNonNullableInput) { - // scale: -2, 0, 5 - // [1, 2, 3], [1, null, 3], [1], [null], [null, null], [] - // values: {1, 2, 3, 1, XXX, 3, 1, XXX, XXX, XXX} - // offsets: {0, 3, 6, 7, 8, 10, 10} - // nullmask: {1, 1, 1, 1, 0, 1, 1, 0, 0, 0} - // outputs: {1, XXX, 1, XXX, XXX, XXX} - // output nullmask: {1, 0, 1, 0, 0, 0} - using RepType = cudf::device_storage_type_t; + auto const offsets = std::vector{0, 3, 4, 4}; + auto const d_offsets = + cudf::detail::make_device_uvector_async(offsets, cudf::get_default_stream()); + auto const agg = cudf::make_min_aggregation(); + for (auto scale : {-2, 0, 5}) { auto const input = - cudf::test::fixed_point_column_wrapper({1, 2, 3, 1, XXX, 3, 1, XXX, XXX, XXX}, - {1, 1, 1, 1, 0, 1, 1, 0, 0, 0}, - numeric::scale_type{scale}); - auto const offsets = std::vector{0, 3, 6, 7, 8, 10, 10}; - auto const d_offsets = - cudf::detail::make_device_uvector_async(offsets, cudf::get_default_stream()); + cudf::test::fixed_point_column_wrapper({1, 2, 3, 1}, numeric::scale_type{scale}); auto out_type = cudf::column_view(input).type(); auto const expect = cudf::test::fixed_point_column_wrapper( - {1, XXX, 1, XXX, XXX, XXX}, {1, 0, 1, 0, 0, 0}, numeric::scale_type{scale}); + {1, 1, XXX}, {1, 1, 0}, numeric::scale_type{scale}); - auto res = - cudf::segmented_reduce(input, - d_offsets, - *cudf::make_min_aggregation(), - out_type, - cudf::null_policy::INCLUDE); + auto result = + cudf::segmented_reduce(input, d_offsets, *agg, out_type, cudf::null_policy::INCLUDE); + CUDF_TEST_EXPECT_COLUMNS_EQUAL(*result, expect); - CUDF_TEST_EXPECT_COLUMNS_EQUAL(*res, expect); + result = cudf::segmented_reduce(input, d_offsets, *agg, out_type, cudf::null_policy::EXCLUDE); + CUDF_TEST_EXPECT_COLUMNS_EQUAL(*result, expect); } } -TYPED_TEST(SegmentedReductionFixedPointTest, MinExcludeNulls) +TYPED_TEST(SegmentedReductionFixedPointTest, Sum) { - // scale: -2, 0, 5 - // [1, 2, 3], [1, null, 3], [1], [null], [null, null], [] - // values: {1, 2, 3, 1, XXX, 3, 1, XXX, XXX, XXX} - // offsets: {0, 3, 6, 7, 8, 10, 10} - // nullmask: {1, 1, 1, 1, 0, 1, 1, 0, 0, 0} - // outputs: {1, 1, 1, XXX, XXX, XXX} - // output nullmask: {1, 1, 1, 0, 0, 0} - using RepType = cudf::device_storage_type_t; + auto const offsets = std::vector{0, 3, 6, 7, 8, 10, 10}; + auto const d_offsets = + cudf::detail::make_device_uvector_async(offsets, cudf::get_default_stream()); + auto const agg = cudf::make_sum_aggregation(); + for (auto scale : {-2, 0, 5}) { - auto const input = - cudf::test::fixed_point_column_wrapper({1, 2, 3, 1, XXX, 3, 1, XXX, XXX, XXX}, + auto input = + cudf::test::fixed_point_column_wrapper({-10, 0, 33, 100, XXX, 53, 11, XXX, XXX, XXX}, {1, 1, 1, 1, 0, 1, 1, 0, 0, 0}, numeric::scale_type{scale}); - auto const offsets = std::vector{0, 3, 6, 7, 8, 10, 10}; - auto const d_offsets = - cudf::detail::make_device_uvector_async(offsets, cudf::get_default_stream()); - auto out_type = cudf::column_view(input).type(); - auto const expect = cudf::test::fixed_point_column_wrapper( - {1, 1, 1, XXX, XXX, XXX}, {1, 1, 1, 0, 0, 0}, numeric::scale_type{scale}); + auto const out_type = cudf::column_view(input).type(); + + auto expect = cudf::test::fixed_point_column_wrapper( + {23, XXX, 11, XXX, XXX, XXX}, {1, 0, 1, 0, 0, 0}, numeric::scale_type{scale}); + auto result = + cudf::segmented_reduce(input, d_offsets, *agg, out_type, cudf::null_policy::INCLUDE); + CUDF_TEST_EXPECT_COLUMNS_EQUAL(*result, expect); - auto res = - cudf::segmented_reduce(input, - d_offsets, - *cudf::make_min_aggregation(), - out_type, - cudf::null_policy::EXCLUDE); + expect = cudf::test::fixed_point_column_wrapper( + {23, 153, 11, XXX, XXX, XXX}, {1, 1, 1, 0, 0, 0}, numeric::scale_type{scale}); + result = cudf::segmented_reduce(input, d_offsets, *agg, out_type, cudf::null_policy::EXCLUDE); + CUDF_TEST_EXPECT_COLUMNS_EQUAL(*result, expect); - CUDF_TEST_EXPECT_COLUMNS_EQUAL(*res, expect); + input = cudf::test::fixed_point_column_wrapper( + {-10, 0, 33, 100, 123, 53, 11, 0, -120, 88}, numeric::scale_type{scale}); + expect = cudf::test::fixed_point_column_wrapper( + {23, 276, 11, 0, -32, XXX}, {1, 1, 1, 1, 1, 0}, numeric::scale_type{scale}); + result = cudf::segmented_reduce(input, d_offsets, *agg, out_type, cudf::null_policy::INCLUDE); + CUDF_TEST_EXPECT_COLUMNS_EQUAL(*result, expect); + result = cudf::segmented_reduce(input, d_offsets, *agg, out_type, cudf::null_policy::EXCLUDE); + CUDF_TEST_EXPECT_COLUMNS_EQUAL(*result, expect); } } -TYPED_TEST(SegmentedReductionFixedPointTest, MaxNonNullableInput) +TYPED_TEST(SegmentedReductionFixedPointTest, Product) { - // scale: -2, 0, 5 - // [1, 2, 3], [1], [] - // values: {1, 2, 3, 1} - // offsets: {0, 3, 4} - // outputs: {3, 1, XXX} - // output nullmask: {1, 1, 0} - using RepType = cudf::device_storage_type_t; + auto const offsets = std::vector{0, 3, 6, 7, 8, 12, 12}; + auto const d_offsets = + cudf::detail::make_device_uvector_async(offsets, cudf::get_default_stream()); + auto const agg = cudf::make_product_aggregation(); + for (auto scale : {-2, 0, 5}) { - auto const input = - cudf::test::fixed_point_column_wrapper({1, 2, 3, 1}, numeric::scale_type{scale}); - auto const offsets = std::vector{0, 3, 4, 4}; - auto const d_offsets = - cudf::detail::make_device_uvector_async(offsets, cudf::get_default_stream()); - auto out_type = cudf::column_view(input).type(); - auto const expect = cudf::test::fixed_point_column_wrapper( - {3, 1, XXX}, {1, 1, 0}, numeric::scale_type{scale}); + auto input = cudf::test::fixed_point_column_wrapper( + {-10, 1, 33, 40, XXX, 50, 11000, XXX, XXX, XXX, XXX, XXX}, + {1, 1, 1, 1, 0, 1, 1, 0, 0, 0, 0, 0}, + numeric::scale_type{scale}); + auto const out_type = cudf::column_view(input).type(); + auto result = + cudf::segmented_reduce(input, d_offsets, *agg, out_type, cudf::null_policy::INCLUDE); + auto expect = cudf::test::fixed_point_column_wrapper( + {-330, XXX, 11000, XXX, XXX, XXX}, {1, 0, 1, 0, 0, 0}, numeric::scale_type{scale * 3}); + CUDF_TEST_EXPECT_COLUMNS_EQUAL(*result, expect); + + result = cudf::segmented_reduce(input, d_offsets, *agg, out_type, cudf::null_policy::EXCLUDE); + expect = cudf::test::fixed_point_column_wrapper( + {-330, 2000, 11000, XXX, XXX, XXX}, {1, 1, 1, 0, 0, 0}, numeric::scale_type{scale * 3}); + CUDF_TEST_EXPECT_COLUMNS_EQUAL(*result, expect); - auto include_null_res = - cudf::segmented_reduce(input, - d_offsets, - *cudf::make_max_aggregation(), - out_type, - cudf::null_policy::INCLUDE); - - auto exclude_null_res = - cudf::segmented_reduce(input, - d_offsets, - *cudf::make_max_aggregation(), - out_type, - cudf::null_policy::EXCLUDE); - - CUDF_TEST_EXPECT_COLUMNS_EQUAL(*include_null_res, expect); - CUDF_TEST_EXPECT_COLUMNS_EQUAL(*exclude_null_res, expect); + input = cudf::test::fixed_point_column_wrapper( + {-10, 1, 33, 3, 40, 50, 11000, 0, -3, 50, 10, 4}, numeric::scale_type{scale}); + expect = cudf::test::fixed_point_column_wrapper( + {-330, 6000, 11000, 0, -6000, XXX}, {1, 1, 1, 1, 1, 0}, numeric::scale_type{scale * 4}); + result = cudf::segmented_reduce(input, d_offsets, *agg, out_type, cudf::null_policy::INCLUDE); + CUDF_TEST_EXPECT_COLUMNS_EQUAL(*result, expect); + result = cudf::segmented_reduce(input, d_offsets, *agg, out_type, cudf::null_policy::EXCLUDE); + CUDF_TEST_EXPECT_COLUMNS_EQUAL(*result, expect); } } -TYPED_TEST(SegmentedReductionFixedPointTest, MinNonNullableInput) +TYPED_TEST(SegmentedReductionFixedPointTest, SumOfSquares) { - // scale: -2, 0, 5 - // [1, 2, 3], [1], [] - // values: {1, 2, 3, 1} - // offsets: {0, 3, 4} - // outputs: {1, 1, XXX} - // output nullmask: {1, 1, 0} - using RepType = cudf::device_storage_type_t; + auto const offsets = std::vector{0, 3, 6, 7, 8, 10, 10}; + auto const d_offsets = + cudf::detail::make_device_uvector_async(offsets, cudf::get_default_stream()); + auto const agg = cudf::make_sum_of_squares_aggregation(); + for (auto scale : {-2, 0, 5}) { - auto const input = - cudf::test::fixed_point_column_wrapper({1, 2, 3, 1}, numeric::scale_type{scale}); - auto const offsets = std::vector{0, 3, 4, 4}; - auto const d_offsets = - cudf::detail::make_device_uvector_async(offsets, cudf::get_default_stream()); - auto out_type = cudf::column_view(input).type(); - auto const expect = cudf::test::fixed_point_column_wrapper( - {1, 1, XXX}, {1, 1, 0}, numeric::scale_type{scale}); + auto input = + cudf::test::fixed_point_column_wrapper({-10, 0, 33, 100, XXX, 53, 11, XXX, XXX, XXX}, + {1, 1, 1, 1, 0, 1, 1, 0, 0, 0}, + numeric::scale_type{scale}); + auto const out_type = cudf::column_view(input).type(); + + auto expect = cudf::test::fixed_point_column_wrapper( + {1189, XXX, 121, XXX, XXX, XXX}, {1, 0, 1, 0, 0, 0}, numeric::scale_type{scale * 2}); + auto result = + cudf::segmented_reduce(input, d_offsets, *agg, out_type, cudf::null_policy::INCLUDE); + CUDF_TEST_EXPECT_COLUMNS_EQUAL(*result, expect); - auto include_null_res = - cudf::segmented_reduce(input, - d_offsets, - *cudf::make_min_aggregation(), - out_type, - cudf::null_policy::INCLUDE); - - auto exclude_null_res = - cudf::segmented_reduce(input, - d_offsets, - *cudf::make_min_aggregation(), - out_type, - cudf::null_policy::EXCLUDE); - - CUDF_TEST_EXPECT_COLUMNS_EQUAL(*include_null_res, expect); - CUDF_TEST_EXPECT_COLUMNS_EQUAL(*exclude_null_res, expect); + expect = cudf::test::fixed_point_column_wrapper( + {1189, 12809, 121, XXX, XXX, XXX}, {1, 1, 1, 0, 0, 0}, numeric::scale_type{scale * 2}); + result = cudf::segmented_reduce(input, d_offsets, *agg, out_type, cudf::null_policy::EXCLUDE); + CUDF_TEST_EXPECT_COLUMNS_EQUAL(*result, expect); + + input = cudf::test::fixed_point_column_wrapper( + {-10, 0, 33, 100, 123, 53, 11, 0, -120, 88}, numeric::scale_type{scale}); + expect = cudf::test::fixed_point_column_wrapper( + {1189, 27938, 121, 0, 22144, XXX}, {1, 1, 1, 1, 1, 0}, numeric::scale_type{scale * 2}); + result = cudf::segmented_reduce(input, d_offsets, *agg, out_type, cudf::null_policy::INCLUDE); + CUDF_TEST_EXPECT_COLUMNS_EQUAL(*result, expect); + result = cudf::segmented_reduce(input, d_offsets, *agg, out_type, cudf::null_policy::EXCLUDE); + CUDF_TEST_EXPECT_COLUMNS_EQUAL(*result, expect); } } From fffdc0cdac913d402c91880919ee391877f47514 Mon Sep 17 00:00:00 2001 From: David Wendt <45795991+davidwendt@users.noreply.github.com> Date: Wed, 22 Feb 2023 20:52:42 -0500 Subject: [PATCH 05/60] Add compute-sanitizer github workflow action to nightly tests (#12800) Adds github workflow action to the nightly tests for running `compute-sanitizer` on the libcudf gtests. Reference: #12530 Authors: - David Wendt (https://github.com/davidwendt) - AJ Schmidt (https://github.com/ajschmidt8) Approvers: - AJ Schmidt (https://github.com/ajschmidt8) - Vyas Ramasubramani (https://github.com/vyasr) - Bradley Dice (https://github.com/bdice) URL: https://github.com/rapidsai/cudf/pull/12800 --- .github/workflows/test.yaml | 12 +++++++ ci/test_cpp.sh | 30 +---------------- ci/test_cpp_common.sh | 32 +++++++++++++++++++ ci/test_cpp_memcheck.sh | 25 +++++++++++++++ .../all_cuda-118_arch-x86_64.yaml | 1 + dependencies.yaml | 12 +++++++ 6 files changed, 83 insertions(+), 29 deletions(-) create mode 100644 ci/test_cpp_common.sh create mode 100755 ci/test_cpp_memcheck.sh diff --git a/.github/workflows/test.yaml b/.github/workflows/test.yaml index ff19d51f8ef..4d9e97a7b28 100644 --- a/.github/workflows/test.yaml +++ b/.github/workflows/test.yaml @@ -22,6 +22,18 @@ jobs: branch: ${{ inputs.branch }} date: ${{ inputs.date }} sha: ${{ inputs.sha }} + conda-cpp-memcheck-tests: + secrets: inherit + uses: rapidsai/shared-action-workflows/.github/workflows/custom-job.yaml@branch-23.04 + with: + build_type: nightly + branch: ${{ inputs.branch }} + date: ${{ inputs.date }} + sha: ${{ inputs.sha }} + node_type: "gpu-latest-1" + arch: "amd64" + container_image: "rapidsai/ci:latest" + run_script: "ci/test_cpp_memcheck.sh" conda-python-cudf-tests: secrets: inherit uses: rapidsai/shared-action-workflows/.github/workflows/conda-python-tests.yaml@branch-23.04 diff --git a/ci/test_cpp.sh b/ci/test_cpp.sh index 983a63d4ce9..bd7a82afbea 100755 --- a/ci/test_cpp.sh +++ b/ci/test_cpp.sh @@ -1,35 +1,7 @@ #!/bin/bash # Copyright (c) 2022-2023, NVIDIA CORPORATION. -set -euo pipefail - -. /opt/conda/etc/profile.d/conda.sh - -rapids-logger "Generate C++ testing dependencies" -rapids-dependency-file-generator \ - --output conda \ - --file_key test_cpp \ - --matrix "cuda=${RAPIDS_CUDA_VERSION%.*};arch=$(arch)" | tee env.yaml - -rapids-mamba-retry env create --force -f env.yaml -n test - -# Temporarily allow unbound variables for conda activation. -set +u -conda activate test -set -u - -CPP_CHANNEL=$(rapids-download-conda-from-s3 cpp) -RAPIDS_TESTS_DIR=${RAPIDS_TESTS_DIR:-"${PWD}/test-results"}/ -mkdir -p "${RAPIDS_TESTS_DIR}" - -rapids-print-env - -rapids-mamba-retry install \ - --channel "${CPP_CHANNEL}" \ - libcudf libcudf_kafka libcudf-tests - -rapids-logger "Check GPU usage" -nvidia-smi +source "$(dirname "$0")/test_cpp_common.sh" EXITCODE=0 trap "EXITCODE=1" ERR diff --git a/ci/test_cpp_common.sh b/ci/test_cpp_common.sh new file mode 100644 index 00000000000..c7c095dc4df --- /dev/null +++ b/ci/test_cpp_common.sh @@ -0,0 +1,32 @@ +#!/bin/bash +# Copyright (c) 2022-2023, NVIDIA CORPORATION. + +set -euo pipefail + +. /opt/conda/etc/profile.d/conda.sh + +rapids-logger "Generate C++ testing dependencies" +rapids-dependency-file-generator \ + --output conda \ + --file_key test_cpp \ + --matrix "cuda=${RAPIDS_CUDA_VERSION%.*};arch=$(arch)" | tee env.yaml + +rapids-mamba-retry env create --force -f env.yaml -n test + +# Temporarily allow unbound variables for conda activation. +set +u +conda activate test +set -u + +CPP_CHANNEL=$(rapids-download-conda-from-s3 cpp) +RAPIDS_TESTS_DIR=${RAPIDS_TESTS_DIR:-"${PWD}/test-results"}/ +mkdir -p "${RAPIDS_TESTS_DIR}" + +rapids-print-env + +rapids-mamba-retry install \ + --channel "${CPP_CHANNEL}" \ + libcudf libcudf_kafka libcudf-tests + +rapids-logger "Check GPU usage" +nvidia-smi diff --git a/ci/test_cpp_memcheck.sh b/ci/test_cpp_memcheck.sh new file mode 100755 index 00000000000..0cad4fc3a3f --- /dev/null +++ b/ci/test_cpp_memcheck.sh @@ -0,0 +1,25 @@ +#!/bin/bash +# Copyright (c) 2023, NVIDIA CORPORATION. + +source "$(dirname "$0")/test_cpp_common.sh" + +EXITCODE=0 +trap "EXITCODE=1" ERR +set +e + +# Run gtests with compute-sanitizer +rapids-logger "Memcheck gtests with rmm_mode=cuda" +export GTEST_CUDF_RMM_MODE=cuda +COMPUTE_SANITIZER_CMD="compute-sanitizer --tool memcheck" +for gt in "$CONDA_PREFIX"/bin/gtests/{libcudf,libcudf_kafka}/* ; do + test_name=$(basename ${gt}) + if [[ "$test_name" == "ERROR_TEST" ]] || [[ "$test_name" == "STREAM_IDENTIFICATION_TEST" ]]; then + continue + fi + echo "Running compute-sanitizer on $test_name" + ${COMPUTE_SANITIZER_CMD} ${gt} --gtest_output=xml:"${RAPIDS_TESTS_DIR}${test_name}.xml" +done +unset GTEST_CUDF_RMM_MODE + +rapids-logger "Test script exiting with value: $EXITCODE" +exit ${EXITCODE} diff --git a/conda/environments/all_cuda-118_arch-x86_64.yaml b/conda/environments/all_cuda-118_arch-x86_64.yaml index 675df3891c3..44d6be65574 100644 --- a/conda/environments/all_cuda-118_arch-x86_64.yaml +++ b/conda/environments/all_cuda-118_arch-x86_64.yaml @@ -16,6 +16,7 @@ dependencies: - cmake>=3.23.1,!=3.25.0 - cubinlinker - cuda-python>=11.7.1,<12.0 +- cuda-sanitizer-api=11.8.86 - cudatoolkit=11.8 - cupy>=9.5.0,<12.0.0a0 - cxx-compiler diff --git a/dependencies.yaml b/dependencies.yaml index ae8eac4ea30..de9795b4b39 100644 --- a/dependencies.yaml +++ b/dependencies.yaml @@ -13,12 +13,14 @@ files: - notebooks - py_version - run + - test_cpp - test_python test_cpp: output: none includes: - cudatoolkit - libidentify_stream_usage_build + - test_cpp test_python: output: none includes: @@ -266,6 +268,16 @@ dependencies: arch: aarch64 packages: - cupy-cuda11x -f https://pip.cupy.dev/aarch64 # TODO: Verify that this works. + test_cpp: + specific: + - output_types: conda + matrices: + - matrix: + cuda: "11.8" + packages: + - cuda-sanitizer-api=11.8.86 + - matrix: + packages: test_java: common: - output_types: conda From 5719463d2db460e5cbbeb266512904dc1b475c2c Mon Sep 17 00:00:00 2001 From: Jake Awe <50372925+AyodeAwe@users.noreply.github.com> Date: Thu, 23 Feb 2023 09:12:50 -0600 Subject: [PATCH 06/60] Add docs build job (#12592) The PR adds a `docs_build` process to the PR and Build workflows for this repository. The generated docs are synced to s3 for only the build workflows. Authors: - Jake Awe (https://github.com/AyodeAwe) - AJ Schmidt (https://github.com/ajschmidt8) Approvers: - AJ Schmidt (https://github.com/ajschmidt8) - Bradley Dice (https://github.com/bdice) URL: https://github.com/rapidsai/cudf/pull/12592 --- .github/workflows/build.yaml | 11 +++++++++ .github/workflows/pr.yaml | 11 +++++++++ .gitignore | 3 +++ ci/build_docs.sh | 47 ++++++++++++++++++++++++++++++++++++ dependencies.yaml | 13 +++++++--- docs/cudf/source/conf.py | 6 ----- 6 files changed, 82 insertions(+), 9 deletions(-) create mode 100755 ci/build_docs.sh diff --git a/.github/workflows/build.yaml b/.github/workflows/build.yaml index fa6704ef04e..024eb828e3c 100644 --- a/.github/workflows/build.yaml +++ b/.github/workflows/build.yaml @@ -53,6 +53,17 @@ jobs: date: ${{ inputs.date }} sha: ${{ inputs.sha }} skip_upload_pkgs: libcudf-example + docs-build: + if: github.ref_type == 'branch' && github.event_name == 'push' + needs: python-build + secrets: inherit + uses: rapidsai/shared-action-workflows/.github/workflows/custom-job.yaml@branch-23.04 + with: + build_type: branch + node_type: "gpu-latest-1" + arch: "amd64" + container_image: "rapidsai/ci:latest" + run_script: "ci/build_docs.sh" wheel-build-cudf: secrets: inherit uses: rapidsai/shared-action-workflows/.github/workflows/wheels-manylinux-build.yml@branch-23.04 diff --git a/.github/workflows/pr.yaml b/.github/workflows/pr.yaml index 73df2de20c2..952b58abda5 100644 --- a/.github/workflows/pr.yaml +++ b/.github/workflows/pr.yaml @@ -20,6 +20,7 @@ jobs: - conda-python-other-tests - conda-java-tests - conda-notebook-tests + - docs-build - wheel-build-cudf - wheel-tests-cudf - wheel-build-dask-cudf @@ -82,6 +83,16 @@ jobs: arch: "amd64" container_image: "rapidsai/ci:latest" run_script: "ci/test_notebooks.sh" + docs-build: + needs: conda-python-build + secrets: inherit + uses: rapidsai/shared-action-workflows/.github/workflows/custom-job.yaml@branch-23.04 + with: + build_type: pull-request + node_type: "gpu-latest-1" + arch: "amd64" + container_image: "rapidsai/ci:latest" + run_script: "ci/build_docs.sh" wheel-build-cudf: needs: checks secrets: inherit diff --git a/.gitignore b/.gitignore index 2d83aad7712..fb5c301fe3f 100644 --- a/.gitignore +++ b/.gitignore @@ -166,6 +166,9 @@ docs/cudf/source/api_docs/generated/* docs/cudf/source/api_docs/api/* docs/cudf/source/user_guide/example_output/* docs/cudf/source/user_guide/cudf.*Dtype.*.rst +_html +_text +jupyter_execute # cibuildwheel /wheelhouse diff --git a/ci/build_docs.sh b/ci/build_docs.sh new file mode 100755 index 00000000000..9551d98e9fe --- /dev/null +++ b/ci/build_docs.sh @@ -0,0 +1,47 @@ +#!/bin/bash +# Copyright (c) 2023, NVIDIA CORPORATION. + +set -euo pipefail + +rapids-logger "Create test conda environment" +. /opt/conda/etc/profile.d/conda.sh + +rapids-dependency-file-generator \ + --output conda \ + --file_key docs \ + --matrix "cuda=${RAPIDS_CUDA_VERSION%.*};arch=$(arch);py=${RAPIDS_PY_VERSION}" | tee env.yaml + +rapids-mamba-retry env create --force -f env.yaml -n docs +conda activate docs + +rapids-print-env + +rapids-logger "Downloading artifacts from previous jobs" +CPP_CHANNEL=$(rapids-download-conda-from-s3 cpp) +PYTHON_CHANNEL=$(rapids-download-conda-from-s3 python) +VERSION_NUMBER=$(rapids-get-rapids-version-from-git) + +rapids-mamba-retry install \ + --channel "${CPP_CHANNEL}" \ + --channel "${PYTHON_CHANNEL}" \ + libcudf cudf dask-cudf + + +rapids-logger "Build Doxygen docs" +pushd cpp/doxygen +aws s3 cp s3://rapidsai-docs/librmm/${VERSION_NUMBER}/html/rmm.tag . || echo "Failed to download rmm Doxygen tag" +doxygen Doxyfile +popd + +rapids-logger "Build Sphinx docs" +pushd docs/cudf +sphinx-build -b dirhtml source _html +sphinx-build -b text source _text +popd + + +if [[ ${RAPIDS_BUILD_TYPE} == "branch" ]]; then + aws s3 sync --delete cpp/doxygen/html "s3://rapidsai-docs/libcudf/${VERSION_NUMBER}/html" + aws s3 sync --delete docs/cudf/_html "s3://rapidsai-docs/cudf/${VERSION_NUMBER}/html" + aws s3 sync --delete docs/cudf/_text "s3://rapidsai-docs/cudf/${VERSION_NUMBER}/txt" +fi diff --git a/dependencies.yaml b/dependencies.yaml index de9795b4b39..ba6d240e069 100644 --- a/dependencies.yaml +++ b/dependencies.yaml @@ -9,7 +9,7 @@ files: - build - cudatoolkit - develop - - doc + - docs - notebooks - py_version - run @@ -43,6 +43,12 @@ files: includes: - develop - py_version + docs: + output: none + includes: + - cudatoolkit + - docs + - py_version channels: - rapidsai - rapidsai-nightly @@ -123,10 +129,11 @@ dependencies: - output_types: conda packages: - doxygen=1.8.20 # pre-commit hook needs a specific version. - doc: + docs: common: - - output_types: [conda, requirements] + - output_types: [conda] packages: + - doxygen=1.8.20 - myst-nb - nbsphinx - numpydoc diff --git a/docs/cudf/source/conf.py b/docs/cudf/source/conf.py index 371a8b4e1c1..3d92d955263 100644 --- a/docs/cudf/source/conf.py +++ b/docs/cudf/source/conf.py @@ -1,4 +1,3 @@ -#!/usr/bin/env python3 # Copyright (c) 2018-2023, NVIDIA CORPORATION. # # cudf documentation build configuration file, created by @@ -23,11 +22,6 @@ from docutils.nodes import Text from sphinx.addnodes import pending_xref -import cudf - -sys.path.insert(0, os.path.abspath(cudf.__path__[0])) -sys.path.insert(0, os.path.abspath(".")) -sys.path.insert(0, os.path.abspath("../..")) sys.path.append(os.path.abspath("./_ext")) # -- General configuration ------------------------------------------------ From 430d91e9acb0a3db7c45df214e5158750311e626 Mon Sep 17 00:00:00 2001 From: Mike Wilson Date: Thu, 23 Feb 2023 13:56:32 -0500 Subject: [PATCH 07/60] Shuffling read into a sub function in parquet read (#12809) This change is the first step toward the pipelined parquet reader and moves the chunk creation and file reads into another function. Right now, the operation is the same, but this change will allow for smaller groups to be read at a time for pipelining. Authors: - Mike Wilson (https://github.com/hyperbolic2346) Approvers: - Nghia Truong (https://github.com/ttnghia) URL: https://github.com/rapidsai/cudf/pull/12809 --- cpp/src/io/parquet/reader_impl.hpp | 15 ++++++- cpp/src/io/parquet/reader_impl_preprocess.cu | 43 ++++++++++++-------- 2 files changed, 39 insertions(+), 19 deletions(-) diff --git a/cpp/src/io/parquet/reader_impl.hpp b/cpp/src/io/parquet/reader_impl.hpp index fcfea35f50c..8b86412ae63 100644 --- a/cpp/src/io/parquet/reader_impl.hpp +++ b/cpp/src/io/parquet/reader_impl.hpp @@ -1,5 +1,5 @@ /* - * Copyright (c) 2019-2022, NVIDIA CORPORATION. + * Copyright (c) 2019-2023, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -130,10 +130,21 @@ class reader::impl { bool uses_custom_row_bounds, host_span const> row_group_indices); + /** + * @brief Create chunk information and start file reads + * + * @param row_groups_info vector of information about row groups to read + * @param num_rows Maximum number of rows to read + * @return pair of boolean indicating if compressed chunks were found and a vector of futures for + * read completion + */ + std::pair>> create_and_read_column_chunks( + cudf::host_span const row_groups_info, size_type num_rows); + /** * @brief Load and decompress the input file(s) into memory. */ - void load_and_decompress_data(std::vector const& row_groups_info, + void load_and_decompress_data(cudf::host_span const row_groups_info, size_type num_rows); /** diff --git a/cpp/src/io/parquet/reader_impl_preprocess.cu b/cpp/src/io/parquet/reader_impl_preprocess.cu index b1d013a96a3..0f55cd6e400 100644 --- a/cpp/src/io/parquet/reader_impl_preprocess.cu +++ b/cpp/src/io/parquet/reader_impl_preprocess.cu @@ -651,16 +651,11 @@ void reader::impl::allocate_nesting_info() page_nesting_decode_info.host_to_device(_stream); } -void reader::impl::load_and_decompress_data(std::vector const& row_groups_info, - size_type num_rows) +std::pair>> reader::impl::create_and_read_column_chunks( + cudf::host_span const row_groups_info, size_type num_rows) { - // This function should never be called if `num_rows == 0`. - CUDF_EXPECTS(num_rows > 0, "Number of reading rows must not be zero."); - - auto& raw_page_data = _file_itm_data.raw_page_data; - auto& decomp_page_data = _file_itm_data.decomp_page_data; - auto& chunks = _file_itm_data.chunks; - auto& pages_info = _file_itm_data.pages_info; + auto& raw_page_data = _file_itm_data.raw_page_data; + auto& chunks = _file_itm_data.chunks; // Descriptors for all the chunks that make up the selected columns const auto num_input_columns = _input_columns.size(); @@ -732,7 +727,7 @@ void reader::impl::load_and_decompress_data(std::vector const& r total_decompressed_size += col_meta.total_uncompressed_size; } } - remaining_rows -= row_group.num_rows; + remaining_rows -= row_group_rows; } // Read compressed chunk data to device memory @@ -745,12 +740,29 @@ void reader::impl::load_and_decompress_data(std::vector const& r chunk_source_map, _stream)); + CUDF_EXPECTS(remaining_rows == 0, "All rows data must be read."); + + return {total_decompressed_size > 0, std::move(read_rowgroup_tasks)}; +} + +void reader::impl::load_and_decompress_data( + cudf::host_span const row_groups_info, size_type num_rows) +{ + // This function should never be called if `num_rows == 0`. + CUDF_EXPECTS(num_rows > 0, "Number of reading rows must not be zero."); + + auto& raw_page_data = _file_itm_data.raw_page_data; + auto& decomp_page_data = _file_itm_data.decomp_page_data; + auto& chunks = _file_itm_data.chunks; + auto& pages_info = _file_itm_data.pages_info; + + auto const [has_compressed_data, read_rowgroup_tasks] = + create_and_read_column_chunks(row_groups_info, num_rows); + for (auto& task : read_rowgroup_tasks) { task.wait(); } - CUDF_EXPECTS(remaining_rows <= 0, "All rows data must be read."); - // Process dataset chunk pages into output columns auto const total_pages = count_page_headers(chunks, _stream); pages_info = hostdevice_vector(total_pages, total_pages, _stream); @@ -758,14 +770,11 @@ void reader::impl::load_and_decompress_data(std::vector const& r if (total_pages > 0) { // decoding of column/page information decode_page_headers(chunks, pages_info, _stream); - if (total_decompressed_size > 0) { + if (has_compressed_data) { decomp_page_data = decompress_page_data(chunks, pages_info, _stream); // Free compressed data for (size_t c = 0; c < chunks.size(); c++) { - if (chunks[c].codec != parquet::Compression::UNCOMPRESSED) { - raw_page_data[c].reset(); - // TODO: Check if this is called - } + if (chunks[c].codec != parquet::Compression::UNCOMPRESSED) { raw_page_data[c].reset(); } } } From e64e26eda09f8508b7760ddba9f742c4f4e827cb Mon Sep 17 00:00:00 2001 From: Ayush Dattagupta Date: Thu, 23 Feb 2023 18:07:56 -0800 Subject: [PATCH 08/60] Expose seed argument to hash_values (#12795) This PR exposes the `seed` param to `hash_values` that is already supported by libcudf's `hash` method. Closes #12775 Authors: - Ayush Dattagupta (https://github.com/ayushdg) Approvers: - https://github.com/brandon-b-miller URL: https://github.com/rapidsai/cudf/pull/12795 --- python/cudf/cudf/core/indexed_frame.py | 24 +++++++++++++-- python/cudf/cudf/tests/test_dataframe.py | 39 +++++++++++++++++++++--- 2 files changed, 57 insertions(+), 6 deletions(-) diff --git a/python/cudf/cudf/core/indexed_frame.py b/python/cudf/cudf/core/indexed_frame.py index 43277fb55ff..2992cb005e5 100644 --- a/python/cudf/cudf/core/indexed_frame.py +++ b/python/cudf/cudf/core/indexed_frame.py @@ -1629,7 +1629,7 @@ def memory_usage(self, index=True, deep=False): """ raise NotImplementedError - def hash_values(self, method="murmur3"): + def hash_values(self, method="murmur3", seed=None): """Compute the hash of values in this column. Parameters @@ -1639,6 +1639,12 @@ def hash_values(self, method="murmur3"): * murmur3: MurmurHash3 hash function. * md5: MD5 hash function. + seed : int, optional + Seed value to use for the hash function. + Note - This only has effect for the following supported + hash functions: + * murmur3: MurmurHash3 hash function. + Returns ------- Series @@ -1665,6 +1671,11 @@ def hash_values(self, method="murmur3"): 1 947ca8d2c5f0f27437f156cfbfab0969 2 d0580ef52d27c043c8e341fd5039b166 dtype: object + >>> series.hash_values(method="murmur3", seed=42) + 0 2364453205 + 1 422621911 + 2 3353449140 + dtype: uint32 **DataFrame** @@ -1686,11 +1697,20 @@ def hash_values(self, method="murmur3"): 2 fe061786ea286a515b772d91b0dfcd70 dtype: object """ + seed_hash_methods = {"murmur3"} + if seed is None: + seed = 0 + elif method not in seed_hash_methods: + warnings.warn( + "Provided seed value has no effect for hash method" + f" `{method}`. Refer to the docstring for information" + " on hash methods that support the `seed` param" + ) # Note that both Series and DataFrame return Series objects from this # calculation, necessitating the unfortunate circular reference to the # child class here. return cudf.Series._from_data( - {None: libcudf.hash.hash([*self._columns], method)}, + {None: libcudf.hash.hash([*self._columns], method, seed)}, index=self.index, ) diff --git a/python/cudf/cudf/tests/test_dataframe.py b/python/cudf/cudf/tests/test_dataframe.py index 09b9f57356c..13f312f6f0c 100644 --- a/python/cudf/cudf/tests/test_dataframe.py +++ b/python/cudf/cudf/tests/test_dataframe.py @@ -38,6 +38,7 @@ NUMERIC_TYPES, assert_eq, assert_exceptions_equal, + assert_neq, does_not_raise, expect_warning_if, gen_rand, @@ -1323,9 +1324,10 @@ def test_assign(): @pytest.mark.parametrize("nrows", [1, 8, 100, 1000]) @pytest.mark.parametrize("method", ["murmur3", "md5"]) -def test_dataframe_hash_values(nrows, method): +@pytest.mark.parametrize("seed", [None, 42]) +def test_dataframe_hash_values(nrows, method, seed): gdf = cudf.DataFrame() - data = np.asarray(range(nrows)) + data = np.arange(nrows) data[0] = data[-1] # make first and last the same gdf["a"] = data gdf["b"] = gdf.a + 100 @@ -1334,12 +1336,41 @@ def test_dataframe_hash_values(nrows, method): assert len(out) == nrows assert out.dtype == np.uint32 + warning_expected = ( + True if seed is not None and method not in {"murmur3"} else False + ) # Check single column - out_one = gdf[["a"]].hash_values(method=method) + if warning_expected: + with pytest.warns( + UserWarning, match="Provided seed value has no effect*" + ): + out_one = gdf[["a"]].hash_values(method=method, seed=seed) + else: + out_one = gdf[["a"]].hash_values(method=method, seed=seed) # First matches last assert out_one.iloc[0] == out_one.iloc[-1] # Equivalent to the cudf.Series.hash_values() - assert_eq(gdf["a"].hash_values(method=method), out_one) + if warning_expected: + with pytest.warns( + UserWarning, match="Provided seed value has no effect*" + ): + assert_eq(gdf["a"].hash_values(method=method, seed=seed), out_one) + else: + assert_eq(gdf["a"].hash_values(method=method, seed=seed), out_one) + + +@pytest.mark.parametrize("method", ["murmur3"]) +def test_dataframe_hash_values_seed(method): + gdf = cudf.DataFrame() + data = np.arange(10) + data[0] = data[-1] # make first and last the same + gdf["a"] = data + gdf["b"] = gdf.a + 100 + out_one = gdf.hash_values(method=method, seed=0) + out_two = gdf.hash_values(method=method, seed=1) + assert out_one.iloc[0] == out_one.iloc[-1] + assert out_two.iloc[0] == out_two.iloc[-1] + assert_neq(out_one, out_two) @pytest.mark.parametrize("nrows", [3, 10, 100, 1000]) From 2e80eba6f75b03f039517c947f386ede65842a4c Mon Sep 17 00:00:00 2001 From: "Richard (Rick) Zamora" Date: Fri, 24 Feb 2023 10:28:25 -0600 Subject: [PATCH 09/60] Fix parquet `RangeIndex` bug (#12838) Possible fix for https://github.com/rapidsai/cudf/issues/12837 Avoids dropping RangeIndex when `columns` argument is passed to `read_parquet` (unless `columns=[]`). Authors: - Richard (Rick) Zamora (https://github.com/rjzamora) Approvers: - GALI PREM SAGAR (https://github.com/galipremsagar) URL: https://github.com/rapidsai/cudf/pull/12838 --- python/cudf/cudf/_lib/parquet.pyx | 2 +- python/cudf/cudf/tests/test_parquet.py | 14 ++++++++++++++ 2 files changed, 15 insertions(+), 1 deletion(-) diff --git a/python/cudf/cudf/_lib/parquet.pyx b/python/cudf/cudf/_lib/parquet.pyx index e5520ae1987..464d9243408 100644 --- a/python/cudf/cudf/_lib/parquet.pyx +++ b/python/cudf/cudf/_lib/parquet.pyx @@ -170,7 +170,7 @@ cpdef read_parquet(filepaths_or_buffers, columns=None, row_groups=None, allow_range_index = True if columns is not None: cpp_columns.reserve(len(columns)) - allow_range_index = False + allow_range_index = len(columns) > 0 for col in columns: cpp_columns.push_back(str(col).encode()) args.set_columns(cpp_columns) diff --git a/python/cudf/cudf/tests/test_parquet.py b/python/cudf/cudf/tests/test_parquet.py index ccd62729a9d..661497e4650 100644 --- a/python/cudf/cudf/tests/test_parquet.py +++ b/python/cudf/cudf/tests/test_parquet.py @@ -2650,6 +2650,20 @@ def test_parquet_columns_and_index_param(index, columns): assert_eq(expected, got, check_index_type=True) +@pytest.mark.parametrize("columns", [None, ["b", "a"]]) +def test_parquet_columns_and_range_index(columns): + buffer = BytesIO() + df = cudf.DataFrame( + {"a": [1, 2, 3], "b": ["a", "b", "c"]}, index=pd.RangeIndex(2, 5) + ) + df.to_parquet(buffer) + + expected = pd.read_parquet(buffer, columns=columns) + got = cudf.read_parquet(buffer, columns=columns) + + assert_eq(expected, got, check_index_type=True) + + def test_parquet_nested_struct_list(): buffer = BytesIO() data = { From 0e4e6dd567964404934d96a1fe8fc14b1d25a526 Mon Sep 17 00:00:00 2001 From: Divye Gala Date: Fri, 24 Feb 2023 12:07:51 -0500 Subject: [PATCH 10/60] Add `always_nullable` flag to Dremel encoding (#12727) Closes #12389 by fixing the bug describe here https://github.com/rapidsai/cudf/issues/12389#issuecomment-1419949751. This flag, when `always_nullable=true`, generates `definition levels` in the Dremel encoding such that it considers every nested column and child to be `nullable`, even if they actually are not. In the context of `two_table_comparators`, this helps us with producing consistently mapped `definition levels` in case there are some nested columns or children that are not nullable in either one or both of the tables. This PR now exposes two APIs: 1. `cudf::detail::get_dremel_data(...)` : This API is consistent with standard Dremel encoding 2. `cudf::detail::get_comparator_data(...)` : This API modifies the definition levels in Dremel encoding to produce the effect described above Authors: - Divye Gala (https://github.com/divyegala) - Nghia Truong (https://github.com/ttnghia) Approvers: - Nghia Truong (https://github.com/ttnghia) - Yunsong Wang (https://github.com/PointKernel) - Bradley Dice (https://github.com/bdice) URL: https://github.com/rapidsai/cudf/pull/12727 --- cpp/include/cudf/lists/detail/dremel.hpp | 30 +++++++-- .../cudf/table/experimental/row_operators.cuh | 3 +- cpp/src/lists/dremel.cu | 48 ++++++++++---- cpp/src/table/row_operators.cu | 2 +- cpp/tests/search/search_list_test.cpp | 64 ++++++++++++++++++- 5 files changed, 124 insertions(+), 23 deletions(-) diff --git a/cpp/include/cudf/lists/detail/dremel.hpp b/cpp/include/cudf/lists/detail/dremel.hpp index 4e3aeec2499..d36a4091947 100644 --- a/cpp/include/cudf/lists/detail/dremel.hpp +++ b/cpp/include/cudf/lists/detail/dremel.hpp @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022, NVIDIA CORPORATION. + * Copyright (c) 2022-2023, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -183,16 +183,34 @@ struct dremel_data { * - | - | -- | --- * ``` * - * @param col Column of LIST type - * @param level_nullability Pre-determined nullability at each list level. Empty means infer from - * `col` + * @param input Column of LIST type + * @param nullability Pre-determined nullability at each list level. Empty means infer from + * `input` + * @param output_as_byte_array if `true`, then any nested list level that has a child of type + * `uint8_t` will be considered as the last level * @param stream CUDA stream used for device memory operations and kernel launches. - * * @return A struct containing dremel data */ -dremel_data get_dremel_data(column_view h_col, +dremel_data get_dremel_data(column_view input, std::vector nullability, bool output_as_byte_array, rmm::cuda_stream_view stream); +/** + * @brief Get Dremel offsets, repetition levels, and modified definition levels to be used for + * lexicographical comparators. The modified definition levels are produced by treating + * each nested column in the input as nullable + * + * @param input Column of LIST type + * @param nullability Pre-determined nullability at each list level. Empty means infer from + * `input` + * @param output_as_byte_array if `true`, then any nested list level that has a child of type + * `uint8_t` will be considered as the last level + * @param stream CUDA stream used for device memory operations and kernel launches. + * @return A struct containing dremel data + */ +dremel_data get_comparator_data(column_view input, + std::vector nullability, + bool output_as_byte_array, + rmm::cuda_stream_view stream); } // namespace cudf::detail diff --git a/cpp/include/cudf/table/experimental/row_operators.cuh b/cpp/include/cudf/table/experimental/row_operators.cuh index f9ffbfcdf7b..2a207d2a5c4 100644 --- a/cpp/include/cudf/table/experimental/row_operators.cuh +++ b/cpp/include/cudf/table/experimental/row_operators.cuh @@ -487,7 +487,8 @@ class device_row_comparator { // element_index because either both rows have a deeply nested NULL at the // same position, and we'll "continue" in our iteration, or we will early // exit if only one of the rows has a deeply nested NULL - if (lcol.nullable() and l_def_levels[l_dremel_index] == l_max_def_level - 1) { + if ((lcol.nullable() and l_def_levels[l_dremel_index] == l_max_def_level - 1) or + (rcol.nullable() and r_def_levels[r_dremel_index] == r_max_def_level - 1)) { ++element_index; } if (l_def_level == r_def_level) { continue; } diff --git a/cpp/src/lists/dremel.cu b/cpp/src/lists/dremel.cu index 26988622aee..c96a21df905 100644 --- a/cpp/src/lists/dremel.cu +++ b/cpp/src/lists/dremel.cu @@ -35,7 +35,7 @@ #include namespace cudf::detail { - +namespace { /** * @brief Functor to get definition level value for a nested struct column until the leaf level or * the first list level. @@ -46,6 +46,7 @@ struct def_level_fn { uint8_t const* d_nullability; uint8_t sub_level_start; uint8_t curr_def_level; + bool always_nullable; __device__ uint32_t operator()(size_type i) { @@ -55,7 +56,7 @@ struct def_level_fn { auto col = *parent_col; do { // If col not nullable then it does not contribute to def levels - if (d_nullability[l]) { + if (always_nullable or d_nullability[l]) { if (not col.nullable() or bit_is_set(col.null_mask(), i)) { ++def; } else { // We have found the shallowest level at which this row is null @@ -72,10 +73,11 @@ struct def_level_fn { } }; -dremel_data get_dremel_data(column_view h_col, - std::vector nullability, - bool output_as_byte_array, - rmm::cuda_stream_view stream) +dremel_data get_encoding(column_view h_col, + std::vector nullability, + bool output_as_byte_array, + bool always_nullable, + rmm::cuda_stream_view stream) { auto get_list_level = [](column_view col) { while (col.type().id() == type_id::STRUCT) { @@ -173,14 +175,14 @@ dremel_data get_dremel_data(column_view h_col, uint32_t def = 0; start_at_sub_level.push_back(curr_nesting_level_idx); while (col.type().id() == type_id::STRUCT) { - def += (nullability[curr_nesting_level_idx]) ? 1 : 0; + def += (always_nullable or nullability[curr_nesting_level_idx]) ? 1 : 0; col = col.child(0); ++curr_nesting_level_idx; } // At the end of all those structs is either a list column or the leaf. List column contributes // at least one def level. Leaf contributes 1 level only if it is nullable. - def += - (col.type().id() == type_id::LIST ? 1 : 0) + (nullability[curr_nesting_level_idx] ? 1 : 0); + def += (col.type().id() == type_id::LIST ? 1 : 0) + + (always_nullable or nullability[curr_nesting_level_idx] ? 1 : 0); def_at_level.push_back(def); ++curr_nesting_level_idx; }; @@ -209,7 +211,7 @@ dremel_data get_dremel_data(column_view h_col, } } - auto [device_view_owners, d_nesting_levels] = + [[maybe_unused]] auto [device_view_owners, d_nesting_levels] = contiguous_copy_column_device_views(nesting_levels, stream); auto max_def_level = def_at_level.back(); @@ -297,7 +299,8 @@ dremel_data get_dremel_data(column_view h_col, def_level_fn{d_nesting_levels + level, d_nullability.data(), start_at_sub_level[level], - def_at_level[level]}); + def_at_level[level], + always_nullable}); // `nesting_levels.size()` == no of list levels + leaf. Max repetition level = no of list levels auto input_child_rep_it = thrust::make_constant_iterator(nesting_levels.size() - 1); @@ -306,7 +309,8 @@ dremel_data get_dremel_data(column_view h_col, def_level_fn{d_nesting_levels + level + 1, d_nullability.data(), start_at_sub_level[level + 1], - def_at_level[level + 1]}); + def_at_level[level + 1], + always_nullable}); // Zip the input and output value iterators so that merge operation is done only once auto input_parent_zip_it = @@ -389,7 +393,8 @@ dremel_data get_dremel_data(column_view h_col, def_level_fn{d_nesting_levels + level, d_nullability.data(), start_at_sub_level[level], - def_at_level[level]}); + def_at_level[level], + always_nullable}); // Zip the input and output value iterators so that merge operation is done only once auto input_parent_zip_it = @@ -459,5 +464,22 @@ dremel_data get_dremel_data(column_view h_col, leaf_data_size, max_def_level}; } +} // namespace + +dremel_data get_dremel_data(column_view h_col, + std::vector nullability, + bool output_as_byte_array, + rmm::cuda_stream_view stream) +{ + return get_encoding(h_col, nullability, output_as_byte_array, false, stream); +} + +dremel_data get_comparator_data(column_view h_col, + std::vector nullability, + bool output_as_byte_array, + rmm::cuda_stream_view stream) +{ + return get_encoding(h_col, nullability, output_as_byte_array, true, stream); +} } // namespace cudf::detail diff --git a/cpp/src/table/row_operators.cu b/cpp/src/table/row_operators.cu index 766a1b63905..8a63a6f6411 100644 --- a/cpp/src/table/row_operators.cu +++ b/cpp/src/table/row_operators.cu @@ -264,7 +264,7 @@ auto list_lex_preprocess(table_view table, rmm::cuda_stream_view stream) std::vector dremel_device_views; for (auto const& col : table) { if (col.type().id() == type_id::LIST) { - dremel_data.push_back(detail::get_dremel_data(col, {}, false, stream)); + dremel_data.push_back(detail::get_comparator_data(col, {}, false, stream)); dremel_device_views.push_back(dremel_data.back()); } } diff --git a/cpp/tests/search/search_list_test.cpp b/cpp/tests/search/search_list_test.cpp index 1393095037d..1e97933fa4d 100644 --- a/cpp/tests/search/search_list_test.cpp +++ b/cpp/tests/search/search_list_test.cpp @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022, NVIDIA CORPORATION. + * Copyright (c) 2022-2023, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -25,7 +25,8 @@ #include #include -using namespace cudf::test::iterators; +using cudf::test::iterators::null_at; +using cudf::test::iterators::nulls_at; using bools_col = cudf::test::fixed_width_column_wrapper; using int32s_col = cudf::test::fixed_width_column_wrapper; @@ -347,3 +348,62 @@ TYPED_TEST(TypedListContainsTestColumnNeedles, ListsOfStructs) auto const result = cudf::contains(*haystack, *needles); CUDF_TEST_EXPECT_COLUMNS_EQUAL(expected, *result, verbosity); } + +struct ListLowerBound : public cudf::test::BaseFixture { +}; + +TEST_F(ListLowerBound, ListWithNulls) +{ + { + using lcw = cudf::test::lists_column_wrapper; + auto const haystack = lcw{ + lcw{-3.45967821e+12}, // 0 + lcw{-3.6912186e-32}, // 1 + lcw{9.721175}, // 2 + }; + + auto const needles = lcw{ + lcw{{0, 4.22671e+32}, null_at(0)}, + }; + + auto const expect = int32s_col{0}; + auto const result = cudf::lower_bound(cudf::table_view{{haystack}}, + cudf::table_view{{needles}}, + {cudf::order::ASCENDING}, + {cudf::null_order::BEFORE}); + CUDF_TEST_EXPECT_COLUMNS_EQUAL(expect, *result); + } + + { + using lcw = cudf::test::lists_column_wrapper; + auto const col1 = lcw{ + lcw{{0}, null_at(0)}, // 0 + lcw{-80}, // 1 + lcw{-17}, // 2 + }; + + auto const col2 = lcw{ + lcw{27}, // 0 + lcw{{0}, null_at(0)}, // 1 + lcw{}, // 2 + }; + + auto const val1 = lcw{ + lcw{87}, + }; + + auto const val2 = lcw{ + lcw{}, + }; + + cudf::table_view input{{col1, col2}}; + cudf::table_view values{{val1, val2}}; + std::vector column_order{cudf::order::ASCENDING, cudf::order::DESCENDING}; + std::vector null_order_flags{cudf::null_order::BEFORE, + cudf::null_order::BEFORE}; + + auto const expect = int32s_col{3}; + auto const result = cudf::lower_bound(input, values, column_order, null_order_flags); + CUDF_TEST_EXPECT_COLUMNS_EQUAL(expect, *result); + } +} From 8a7fb2f14a73937d31f648a65f57bc47751e97c1 Mon Sep 17 00:00:00 2001 From: GALI PREM SAGAR Date: Fri, 24 Feb 2023 12:25:49 -0600 Subject: [PATCH 11/60] Deprecate `inplace` parameters in categorical methods (#12824) To get ready for pandas-2.0 compatibility, this PR deprecates `inplace` in the following APIs: - [x] `as_ordered` - [x] `as_unordered` - [x] `add_categories` - [x] `remove_categories` - [x] `set_categories` - [x] `reorder_categories` Authors: - GALI PREM SAGAR (https://github.com/galipremsagar) Approvers: - https://github.com/brandon-b-miller URL: https://github.com/rapidsai/cudf/pull/12824 --- python/cudf/cudf/core/column/categorical.py | 78 ++++++++++++++++++++- python/cudf/cudf/tests/test_categorical.py | 19 +++-- 2 files changed, 92 insertions(+), 5 deletions(-) diff --git a/python/cudf/cudf/core/column/categorical.py b/python/cudf/cudf/core/column/categorical.py index a1526d25512..52f7c0b957f 100644 --- a/python/cudf/cudf/core/column/categorical.py +++ b/python/cudf/cudf/core/column/categorical.py @@ -141,6 +141,13 @@ def as_ordered(self, inplace: bool = False) -> Optional[SeriesOrIndex]: or return a copy of this categorical with added categories. + .. deprecated:: 23.02 + + The `inplace` parameter is is deprecated and + will be removed in a future version of cudf. + Setting categories as ordered will always + return a new Categorical object. + Returns ------- Categorical @@ -204,6 +211,13 @@ def as_unordered(self, inplace: bool = False) -> Optional[SeriesOrIndex]: in-place or return a copy of this categorical with ordered set to False. + .. deprecated:: 23.02 + + The `inplace` parameter is is deprecated and + will be removed in a future version of cudf. + Setting categories as unordered will always + return a new Categorical object. + Returns ------- Categorical @@ -286,6 +300,13 @@ def add_categories( or return a copy of this categorical with added categories. + .. deprecated:: 23.04 + + The `inplace` parameter is is deprecated and + will be removed in a future version of cudf. + Adding categories will always return a + new Categorical object. + Returns ------- cat @@ -318,7 +339,14 @@ def add_categories( dtype: category Categories (5, int64): [1, 2, 0, 3, 4] """ - + if inplace: + warnings.warn( + "The `inplace` parameter in cudf.Series.cat.add_categories " + "is deprecated and will be removed in a future version of " + "cudf. Adding categories will always return a new " + "Categorical object.", + FutureWarning, + ) old_categories = self._column.categories new_categories = column.as_column( new_categories, @@ -371,6 +399,13 @@ def remove_categories( inplace or return a copy of this categorical with removed categories. + .. deprecated:: 23.04 + + The `inplace` parameter is is deprecated and + will be removed in a future version of cudf. + Removing categories will always return a + new Categorical object. + Returns ------- cat @@ -423,6 +458,16 @@ def remove_categories( dtype: category Categories (2, int64): [1, 2] """ + if inplace: + warnings.warn( + "The `inplace` parameter in " + "cudf.Series.cat.remove_categories is deprecated and " + "will be removed in a future version of cudf. " + "Removing categories will always return a new " + "Categorical object.", + FutureWarning, + ) + cats = self.categories.to_series() removals = cudf.Series(removals, dtype=cats.dtype) removals_mask = removals.isin(cats) @@ -485,6 +530,13 @@ def set_categories( or return a copy of this categorical with reordered categories. + .. deprecated:: 23.04 + + The `inplace` parameter is is deprecated and + will be removed in a future version of cudf. + Setting categories will always return a + new Categorical object. + Returns ------- cat @@ -524,6 +576,14 @@ def set_categories( dtype: category Categories (2, int64): [1, 10] """ + if inplace: + warnings.warn( + "The `inplace` parameter in cudf.Series.cat.set_categories is " + "deprecated and will be removed in a future version of cudf. " + "Setting categories will always return a new Categorical " + "object.", + FutureWarning, + ) return self._return_or_inplace( self._column.set_categories( new_categories=new_categories, ordered=ordered, rename=rename @@ -556,6 +616,13 @@ def reorder_categories( inplace or return a copy of this categorical with reordered categories. + .. deprecated:: 23.04 + + The `inplace` parameter is is deprecated and + will be removed in a future version of cudf. + Reordering categories will always return a + new Categorical object. + Returns ------- cat @@ -597,6 +664,15 @@ def reorder_categories( ValueError: items in new_categories are not the same as in old categories """ + if inplace: + warnings.warn( + "The `inplace` parameter in " + "cudf.Series.cat.reorder_categories is deprecated " + "and will be removed in a future version of cudf. " + "Reordering categories will always return a new " + "Categorical object.", + FutureWarning, + ) return self._return_or_inplace( self._column.reorder_categories(new_categories, ordered=ordered), inplace=inplace, diff --git a/python/cudf/cudf/tests/test_categorical.py b/python/cudf/cudf/tests/test_categorical.py index fa8981cf7e3..496039ca2f8 100644 --- a/python/cudf/cudf/tests/test_categorical.py +++ b/python/cudf/cudf/tests/test_categorical.py @@ -443,10 +443,13 @@ def test_categorical_reorder_categories( "reorder_categories" ): pd_sr_1 = pd_sr.cat.reorder_categories(list("cba"), **kwargs) - cd_sr_1 = cd_sr.cat.reorder_categories(list("cba"), **kwargs) if inplace: + with pytest.warns(FutureWarning): + cd_sr_1 = cd_sr.cat.reorder_categories(list("cba"), **kwargs) pd_sr_1 = pd_sr cd_sr_1 = cd_sr + else: + cd_sr_1 = cd_sr.cat.reorder_categories(list("cba"), **kwargs) assert_eq(pd_sr_1, cd_sr_1) @@ -479,10 +482,14 @@ def test_categorical_add_categories(pd_str_cat, inplace): "add_categories" ): pd_sr_1 = pd_sr.cat.add_categories(["d"], inplace=inplace) - cd_sr_1 = cd_sr.cat.add_categories(["d"], inplace=inplace) + if inplace: + with pytest.warns(FutureWarning): + cd_sr_1 = cd_sr.cat.add_categories(["d"], inplace=inplace) pd_sr_1 = pd_sr cd_sr_1 = cd_sr + else: + cd_sr_1 = cd_sr.cat.add_categories(["d"], inplace=inplace) assert "d" in pd_sr_1.cat.categories.to_list() assert "d" in cd_sr_1.cat.categories.to_pandas().to_list() @@ -516,10 +523,14 @@ def test_categorical_remove_categories(pd_str_cat, inplace): "remove_categories" ): pd_sr_1 = pd_sr.cat.remove_categories(["a"], inplace=inplace) - cd_sr_1 = cd_sr.cat.remove_categories(["a"], inplace=inplace) + if inplace: + with pytest.warns(FutureWarning): + cd_sr_1 = cd_sr.cat.remove_categories(["a"], inplace=inplace) pd_sr_1 = pd_sr cd_sr_1 = cd_sr + else: + cd_sr_1 = cd_sr.cat.remove_categories(["a"], inplace=inplace) assert "a" not in pd_sr_1.cat.categories.to_list() assert "a" not in cd_sr_1.cat.categories.to_pandas().to_list() @@ -529,7 +540,7 @@ def test_categorical_remove_categories(pd_str_cat, inplace): # test using ordered operators with _hide_deprecated_pandas_categorical_inplace_warnings( "remove_categories" - ): + ) as _, pytest.warns(FutureWarning) as _: assert_exceptions_equal( lfunc=cd_sr.to_pandas().cat.remove_categories, rfunc=cd_sr.cat.remove_categories, From 54ee14e36157fe63d0eb58ed7ac8bafc2b1e4932 Mon Sep 17 00:00:00 2001 From: Jordan Jacobelli Date: Fri, 24 Feb 2023 19:37:29 +0100 Subject: [PATCH 12/60] Update datasets download URL (#12840) Update datasets download URL to reduce latency and costs Authors: - Jordan Jacobelli (https://github.com/jjacobelli) Approvers: - AJ Schmidt (https://github.com/ajschmidt8) - GALI PREM SAGAR (https://github.com/galipremsagar) URL: https://github.com/rapidsai/cudf/pull/12840 --- python/cudf/cudf/benchmarks/get_datasets.py | 7 ++----- 1 file changed, 2 insertions(+), 5 deletions(-) diff --git a/python/cudf/cudf/benchmarks/get_datasets.py b/python/cudf/cudf/benchmarks/get_datasets.py index f3b66eda512..7090539bcb0 100644 --- a/python/cudf/cudf/benchmarks/get_datasets.py +++ b/python/cudf/cudf/benchmarks/get_datasets.py @@ -1,4 +1,4 @@ -# Copyright (c) 2020, NVIDIA CORPORATION. +# Copyright (c) 2020-2023, NVIDIA CORPORATION. import argparse import os @@ -9,10 +9,7 @@ Dataset = namedtuple("Dataset", ["url", "dir"]) datasets = { "cuio_dataset": Dataset( - ( - "https://rapidsai-data.s3.us-east-2.amazonaws.com/cudf/" - "benchmark/avro_json_datasets.zip" - ), + "https://data.rapids.ai/cudf/benchmark/avro_json_datasets.zip", "cudf/benchmarks/cuio_data/", ), } From 12e4501c49daac3d0e3837a3f65078e63e20b904 Mon Sep 17 00:00:00 2001 From: David Wendt <45795991+davidwendt@users.noreply.github.com> Date: Fri, 24 Feb 2023 13:42:49 -0500 Subject: [PATCH 13/60] Remove KAFKA_HOST_TEST from compute-sanitizer check (#12831) Removes the `KAFKA_HOST_TEST` from the compute-sanitizer memcheck nighly runs. The following error occurs when running this host test. ``` Running compute-sanitizer on KAFKA_HOST_TEST ========= COMPUTE-SANITIZER Running main() from gmock_main.cc [==========] Running 2 tests from 1 test suite. [----------] Global test environment set-up. [----------] 2 tests from KafkaDatasourceTest [ RUN ] KafkaDatasourceTest.MissingGroupID [ OK ] KafkaDatasourceTest.MissingGroupID (0 ms) [ RUN ] KafkaDatasourceTest.InvalidConfigValues [ OK ] KafkaDatasourceTest.InvalidConfigValues (0 ms) [----------] 2 tests from KafkaDatasourceTest (0 ms total) [----------] Global test environment tear-down [==========] 2 tests from 1 test suite ran. (0 ms total) [ PASSED ] 2 tests. ========= Error: Target application terminated before first instrumented API call ========= Tracking kernels launched by child processes requires the --target-processes all option. ``` Adding the `--target-processes all` option gives the same error. Disabling the check of this test since it is a host test that checks error conditions and does not appear to make any device calls. Authors: - David Wendt (https://github.com/davidwendt) Approvers: - AJ Schmidt (https://github.com/ajschmidt8) - Bradley Dice (https://github.com/bdice) URL: https://github.com/rapidsai/cudf/pull/12831 --- ci/test_cpp_memcheck.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ci/test_cpp_memcheck.sh b/ci/test_cpp_memcheck.sh index 0cad4fc3a3f..db9ce143d51 100755 --- a/ci/test_cpp_memcheck.sh +++ b/ci/test_cpp_memcheck.sh @@ -11,7 +11,7 @@ set +e rapids-logger "Memcheck gtests with rmm_mode=cuda" export GTEST_CUDF_RMM_MODE=cuda COMPUTE_SANITIZER_CMD="compute-sanitizer --tool memcheck" -for gt in "$CONDA_PREFIX"/bin/gtests/{libcudf,libcudf_kafka}/* ; do +for gt in "$CONDA_PREFIX"/bin/gtests/libcudf/* ; do test_name=$(basename ${gt}) if [[ "$test_name" == "ERROR_TEST" ]] || [[ "$test_name" == "STREAM_IDENTIFICATION_TEST" ]]; then continue From 77c2e03ec572527b5c5c7a3f7a48b0cabd29abde Mon Sep 17 00:00:00 2001 From: Vyas Ramasubramani Date: Fri, 24 Feb 2023 10:47:44 -0800 Subject: [PATCH 14/60] Consolidate linter configs into pyproject.toml (#12834) This consolidation allows us to get rid of now unnecessary setup.cfg files (thanks to removing versioneer in #12741). It also allows us to move towards a fully pyproject.toml-driven build. Authors: - Vyas Ramasubramani (https://github.com/vyasr) Approvers: - AJ Schmidt (https://github.com/ajschmidt8) - David Wendt (https://github.com/davidwendt) - Lawrence Mitchell (https://github.com/wence-) URL: https://github.com/rapidsai/cudf/pull/12834 --- .flake8 | 24 +++++++ .pre-commit-config.yaml | 12 ++-- ci/release/update-version.sh | 2 +- cpp/benchmarks/common/generate_input.cu | 4 +- cpp/benchmarks/common/generate_input.hpp | 6 +- .../developer_guide/contributing_guide.md | 8 +-- pyproject.toml | 38 +++++++++++ python/cudf/cudf/_lib/utils.pyx | 4 +- python/cudf/pyproject.toml | 43 +++++++++++++ python/cudf/setup.cfg | 32 ---------- python/cudf_kafka/pyproject.toml | 46 +++++++++++++ python/cudf_kafka/setup.cfg | 35 ---------- python/custreamz/pyproject.toml | 45 +++++++++++++ python/custreamz/setup.cfg | 34 ---------- python/dask_cudf/pyproject.toml | 45 +++++++++++++ python/dask_cudf/setup.cfg | 31 --------- setup.cfg | 64 ------------------- 17 files changed, 261 insertions(+), 212 deletions(-) create mode 100644 .flake8 delete mode 100644 python/cudf/setup.cfg delete mode 100644 python/cudf_kafka/setup.cfg delete mode 100644 python/custreamz/setup.cfg delete mode 100644 setup.cfg diff --git a/.flake8 b/.flake8 new file mode 100644 index 00000000000..e80e3afc443 --- /dev/null +++ b/.flake8 @@ -0,0 +1,24 @@ +# Copyright (c) 2017-2023, NVIDIA CORPORATION. + +[flake8] +filename = *.py, *.pyx, *.pxd, *.pxi +exclude = __init__.py, *.egg, build, docs, .git +force-check = True +ignore = + # line break before binary operator + W503, + # whitespace before : + E203 +per-file-ignores = + # Rules ignored only in Cython: + # E211: whitespace before '(' (used in multi-line imports) + # E225: Missing whitespace around operators (breaks cython casting syntax like ) + # E226: Missing whitespace around arithmetic operators (breaks cython pointer syntax like int*) + # E227: Missing whitespace around bitwise or shift operator (Can also break casting syntax) + # E275: Missing whitespace after keyword (Doesn't work with Cython except?) + # E402: invalid syntax (works for Python, not Cython) + # E999: invalid syntax (works for Python, not Cython) + # W504: line break after binary operator (breaks lines that end with a pointer) + *.pyx: E211, E225, E226, E227, E275, E402, E999, W504 + *.pxd: E211, E225, E226, E227, E275, E402, E999, W504 + *.pxi: E211, E225, E226, E227, E275, E402, E999, W504 diff --git a/.pre-commit-config.yaml b/.pre-commit-config.yaml index 244fc0d3872..e252af717ce 100644 --- a/.pre-commit-config.yaml +++ b/.pre-commit-config.yaml @@ -34,7 +34,7 @@ repos: rev: 5.0.4 hooks: - id: flake8 - args: ["--config=setup.cfg"] + args: ["--config=.flake8"] files: python/.*$ types: [file] types_or: [python, cython] @@ -48,7 +48,7 @@ repos: hooks: - id: mypy additional_dependencies: [types-cachetools] - args: ["--config-file=setup.cfg", + args: ["--config-file=pyproject.toml", "python/cudf/cudf", "python/custreamz/custreamz", "python/cudf_kafka/cudf_kafka", @@ -58,7 +58,9 @@ repos: rev: 6.1.1 hooks: - id: pydocstyle - args: ["--config=setup.cfg"] + # https://github.com/PyCQA/pydocstyle/issues/603 + additional_dependencies: [toml] + args: ["--config=pyproject.toml"] - repo: https://github.com/pre-commit/mirrors-clang-format rev: v11.1.0 hooks: @@ -138,9 +140,11 @@ repos: pass_filenames: false verbose: false - repo: https://github.com/codespell-project/codespell - rev: v2.1.0 + rev: v2.2.2 hooks: - id: codespell + additional_dependencies: [tomli] + args: ["--toml", "pyproject.toml"] exclude: | (?x)^( .*test.*| diff --git a/ci/release/update-version.sh b/ci/release/update-version.sh index c8875fda641..831b91bb2a6 100755 --- a/ci/release/update-version.sh +++ b/ci/release/update-version.sh @@ -94,7 +94,7 @@ sed_runner "s/rmm==.*\",/rmm==${NEXT_SHORT_TAG_PEP440}.*\",/g" python/cudf/setup sed_runner "s/cudf==.*\",/cudf==${NEXT_SHORT_TAG_PEP440}.*\",/g" python/dask_cudf/setup.py # Dependency versions in pyproject.toml -sed_runner "s/rmm==.*\",/rmm==${NEXT_SHORT_TAG_PEP440}.*\",/g" python/pyproject.toml +sed_runner "s/rmm==.*\",/rmm==${NEXT_SHORT_TAG_PEP440}.*\",/g" python/cudf/pyproject.toml for FILE in .github/workflows/*.yaml; do sed_runner "/shared-action-workflows/ s/@.*/@branch-${NEXT_SHORT_TAG}/g" "${FILE}" diff --git a/cpp/benchmarks/common/generate_input.cu b/cpp/benchmarks/common/generate_input.cu index dee7e2b8586..2829d14070c 100644 --- a/cpp/benchmarks/common/generate_input.cu +++ b/cpp/benchmarks/common/generate_input.cu @@ -1,5 +1,5 @@ /* - * Copyright (c) 2020-2022, NVIDIA CORPORATION. + * Copyright (c) 2020-2023, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -501,7 +501,7 @@ std::unique_ptr create_random_utf8_string_column(data_profile cons rmm::device_uvector offsets(num_rows + 1, cudf::get_default_stream()); thrust::exclusive_scan( thrust::device, valid_lengths, valid_lengths + lengths.size(), offsets.begin()); - // offfsets are ready. + // offsets are ready. auto chars_length = *thrust::device_pointer_cast(offsets.end() - 1); rmm::device_uvector chars(chars_length, cudf::get_default_stream()); thrust::for_each_n(thrust::device, diff --git a/cpp/benchmarks/common/generate_input.hpp b/cpp/benchmarks/common/generate_input.hpp index f8ea194f0c4..e65aa69763b 100644 --- a/cpp/benchmarks/common/generate_input.hpp +++ b/cpp/benchmarks/common/generate_input.hpp @@ -1,5 +1,5 @@ /* - * Copyright (c) 2020-2022, NVIDIA CORPORATION. + * Copyright (c) 2020-2023, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -373,13 +373,13 @@ class data_profile { void set_bool_probability_true(double p) { - CUDF_EXPECTS(p >= 0. and p <= 1., "probablity must be in range [0...1]"); + CUDF_EXPECTS(p >= 0. and p <= 1., "probability must be in range [0...1]"); bool_probability_true = p; } void set_null_probability(std::optional p) { CUDF_EXPECTS(p.value_or(0.) >= 0. and p.value_or(0.) <= 1., - "probablity must be in range [0...1]"); + "probability must be in range [0...1]"); null_probability = p; } void set_cardinality(cudf::size_type c) { cardinality = c; } diff --git a/docs/cudf/source/developer_guide/contributing_guide.md b/docs/cudf/source/developer_guide/contributing_guide.md index 34071f44914..bb3479cf4c1 100644 --- a/docs/cudf/source/developer_guide/contributing_guide.md +++ b/docs/cudf/source/developer_guide/contributing_guide.md @@ -22,16 +22,16 @@ Specifically, cuDF uses the following tools: In conjunction with [type hints](https://docs.python.org/3/library/typing.html), `mypy` can help catch various bugs that are otherwise difficult to find. - [`pydocstyle`](https://github.com/PyCQA/pydocstyle/) lints docstring style. +- [`codespell`](https://github.com/codespell-project/codespell) finds spelling errors. Linter config data is stored in a number of files. -We generally use `pyproject.toml` over `setup.cfg` and avoid project-specific files (e.g. `setup.cfg` > `python/cudf/setup.cfg`). +We generally use `pyproject.toml` over `setup.cfg` and avoid project-specific files (e.g. `pyproject.toml` > `python/cudf/pyproject.toml`). However, differences between tools and the different packages in the repo result in the following caveats: -- `flake8` has no plans to support `pyproject.toml`, so it must live in `setup.cfg`. +- `flake8` has no plans to support `pyproject.toml`, so it must live in `.flake8`. - `isort` must be configured per project to set which project is the "first party" project. -Additionally, our use of `versioneer` means that each project must have a `setup.cfg`. -As a result, we currently maintain both root and project-level `pyproject.toml` and `setup.cfg` files. +As a result, we currently maintain both root and project-level `pyproject.toml` files as well as a `.flake8` file. For more information on how to use pre-commit hooks, see the code formatting section of the [overall contributing guide](https://github.com/rapidsai/cudf/blob/main/CONTRIBUTING.md#python--pre-commit-hooks). diff --git a/pyproject.toml b/pyproject.toml index dfd22f33785..3940d9119ae 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -17,3 +17,41 @@ force-exclude = ''' dist )/ ''' + +[tool.pydocstyle] +# Due to https://github.com/PyCQA/pydocstyle/issues/363, we must exclude rather +# than include using match-dir. Note that as discussed in +# https://stackoverflow.com/questions/65478393/how-to-filter-directories-using-the-match-dir-flag-for-pydocstyle, +# unlike the match option above this match-dir will have no effect when +# pydocstyle is invoked from pre-commit. Therefore this exclusion list must +# also be maintained in the pre-commit config file. +match-dir = "^(?!(ci|cpp|conda|docs|java|notebooks)).*$" +# Allow missing docstrings for docutils +ignore-decorators = ".*(docutils|doc_apply|copy_docstring).*" +select = "D201, D204, D206, D207, D208, D209, D210, D211, D214, D215, D300, D301, D302, D403, D405, D406, D407, D408, D409, D410, D411, D412, D414, D418" + # Would like to enable the following rules in the future: + # D200, D202, D205, D400 + +[tool.mypy] +ignore_missing_imports = true +# If we don't specify this, then mypy will check excluded files if +# they are imported by a checked file. +follow_imports = "skip" +exclude = [ + "cudf/_lib/", + "cudf/cudf/benchmarks/", + "cudf/cudf/tests/", + "cudf/cudf/utils/metadata/orc_column_statistics_pb2.py", + "custreamz/custreamz/tests/", + "dask_cudf/dask_cudf/tests/", + ] + +[tool.codespell] +# note: pre-commit passes explicit lists of files here, which this skip file list doesn't override - +# this is only to allow you to run codespell interactively +skip = "./.git,./.github,./cpp/build,.*egg-info.*,./.mypy_cache,./cpp/tests,./python/cudf/cudf/tests,./java/src/test,./cpp/include/cudf_test/cxxopts.hpp" +# ignore short words, and typename parameters like OffsetT +ignore-regex = "\\b(.{1,4}|[A-Z]\\w*T)\\b" +ignore-words-list = "inout,unparseable,falsy" +builtin = "clear" +quiet-level = 3 diff --git a/python/cudf/cudf/_lib/utils.pyx b/python/cudf/cudf/_lib/utils.pyx index 5f4d3e17fbc..56918799cca 100644 --- a/python/cudf/cudf/_lib/utils.pyx +++ b/python/cudf/cudf/_lib/utils.pyx @@ -1,4 +1,4 @@ -# Copyright (c) 2020-2022, NVIDIA CORPORATION. +# Copyright (c) 2020-2023, NVIDIA CORPORATION. import numpy as np import pyarrow as pa @@ -315,7 +315,7 @@ cdef columns_from_table_view( object owners, ): """ - Given a ``cudf::table_view``, construsts a list of columns from it, + Given a ``cudf::table_view``, constructs a list of columns from it, along with referencing an owner Python object that owns the memory lifetime. owner must be either None or a list of column. If owner is a list of columns, the owner of the `i`th ``cudf::column_view`` diff --git a/python/cudf/pyproject.toml b/python/cudf/pyproject.toml index 49c4d83245f..305e8822030 100644 --- a/python/cudf/pyproject.toml +++ b/python/cudf/pyproject.toml @@ -15,3 +15,46 @@ requires = [ "protoc-wheel", "rmm==23.4.*", ] + +[tool.isort] +line_length = 79 +multi_line_output = 3 +include_trailing_comma = true +force_grid_wrap = 0 +combine_as_imports = true +order_by_type = true +known_dask = [ + "dask", + "distributed", + "dask_cuda", +] +known_rapids = [ + "rmm", +] +known_first_party = [ + "cudf", +] +default_section = "THIRDPARTY" +sections = [ + "FUTURE", + "STDLIB", + "THIRDPARTY", + "DASK", + "RAPIDS", + "FIRSTPARTY", + "LOCALFOLDER", +] +skip = [ + "thirdparty", + ".eggs", + ".git", + ".hg", + ".mypy_cache", + ".tox", + ".venv", + "_build", + "buck-out", + "build", + "dist", + "__init__.py", +] diff --git a/python/cudf/setup.cfg b/python/cudf/setup.cfg deleted file mode 100644 index 8380da371f9..00000000000 --- a/python/cudf/setup.cfg +++ /dev/null @@ -1,32 +0,0 @@ -# Copyright (c) 2018-2023, NVIDIA CORPORATION. - -[isort] -line_length=79 -multi_line_output=3 -include_trailing_comma=True -force_grid_wrap=0 -combine_as_imports=True -order_by_type=True -known_dask= - dask - distributed - dask_cuda -known_rapids= - rmm -known_first_party= - cudf -default_section=THIRDPARTY -sections=FUTURE,STDLIB,THIRDPARTY,DASK,RAPIDS,FIRSTPARTY,LOCALFOLDER -skip= - thirdparty - .eggs - .git - .hg - .mypy_cache - .tox - .venv - _build - buck-out - build - dist - __init__.py diff --git a/python/cudf_kafka/pyproject.toml b/python/cudf_kafka/pyproject.toml index 0924fc90352..308a7869bc0 100644 --- a/python/cudf_kafka/pyproject.toml +++ b/python/cudf_kafka/pyproject.toml @@ -7,3 +7,49 @@ requires = [ "setuptools", "cython>=0.29,<0.30", ] + +[tool.isort] +line_length = 79 +multi_line_output = 3 +include_trailing_comma = true +force_grid_wrap = 0 +combine_as_imports = true +order_by_type = true +known_dask = [ + "dask", + "distributed", + "dask_cuda", + "streamz", +] +known_rapids = [ + "rmm", + "cudf", + "dask_cudf", +] +known_first_party = [ + "cudf_kafka", +] +default_section = "THIRDPARTY" +sections = [ + "FUTURE", + "STDLIB", + "THIRDPARTY", + "DASK", + "RAPIDS", + "FIRSTPARTY", + "LOCALFOLDER", +] +skip = [ + "thirdparty", + ".eggs", + ".git", + ".hg", + ".mypy_cache", + ".tox", + ".venv", + "_build", + "buck-out", + "build", + "dist", + "__init__.py", +] diff --git a/python/cudf_kafka/setup.cfg b/python/cudf_kafka/setup.cfg deleted file mode 100644 index ee0d783b184..00000000000 --- a/python/cudf_kafka/setup.cfg +++ /dev/null @@ -1,35 +0,0 @@ -# Copyright (c) 2020-2023, NVIDIA CORPORATION. - -[isort] -line_length=79 -multi_line_output=3 -include_trailing_comma=True -force_grid_wrap=0 -combine_as_imports=True -order_by_type=True -known_dask= - dask - distributed - dask_cuda - streamz -known_rapids= - rmm - cudf - dask_cudf -known_first_party= - cudf_kafka -default_section=THIRDPARTY -sections=FUTURE,STDLIB,THIRDPARTY,DASK,RAPIDS,FIRSTPARTY,LOCALFOLDER -skip= - thirdparty - .eggs - .git - .hg - .mypy_cache - .tox - .venv - _build - buck-out - build - dist - __init__.py diff --git a/python/custreamz/pyproject.toml b/python/custreamz/pyproject.toml index 806848c356e..d5c41945482 100644 --- a/python/custreamz/pyproject.toml +++ b/python/custreamz/pyproject.toml @@ -6,3 +6,48 @@ requires = [ "wheel", "setuptools", ] + +[tool.isort] +line_length = 79 +multi_line_output = 3 +include_trailing_comma = true +force_grid_wrap = 0 +combine_as_imports = true +order_by_type = true +known_dask = [ + "dask", + "distributed", + "dask_cuda", +] +known_rapids = [ + "rmm", + "cudf", + "dask_cudf", +] +known_first_party = [ + "streamz", +] +default_section = "THIRDPARTY" +sections = [ + "FUTURE", + "STDLIB", + "THIRDPARTY", + "DASK", + "RAPIDS", + "FIRSTPARTY", + "LOCALFOLDER", +] +skip = [ + "thirdparty", + ".eggs", + ".git", + ".hg", + ".mypy_cache", + ".tox", + ".venv", + "_build", + "buck-out", + "build", + "dist", + "__init__.py", +] diff --git a/python/custreamz/setup.cfg b/python/custreamz/setup.cfg deleted file mode 100644 index 8c038db9349..00000000000 --- a/python/custreamz/setup.cfg +++ /dev/null @@ -1,34 +0,0 @@ -# Copyright (c) 2020-2023, NVIDIA CORPORATION. - -[isort] -line_length=79 -multi_line_output=3 -include_trailing_comma=True -force_grid_wrap=0 -combine_as_imports=True -order_by_type=True -known_dask= - dask - distributed - dask_cuda -known_rapids= - rmm - cudf - dask_cudf -known_first_party= - streamz -default_section=THIRDPARTY -sections=FUTURE,STDLIB,THIRDPARTY,DASK,RAPIDS,FIRSTPARTY,LOCALFOLDER -skip= - thirdparty - .eggs - .git - .hg - .mypy_cache - .tox - .venv - _build - buck-out - build - dist - __init__.py diff --git a/python/dask_cudf/pyproject.toml b/python/dask_cudf/pyproject.toml index 806848c356e..8cf823d4291 100644 --- a/python/dask_cudf/pyproject.toml +++ b/python/dask_cudf/pyproject.toml @@ -6,3 +6,48 @@ requires = [ "wheel", "setuptools", ] + +[tool.isort] +line_length = 79 +multi_line_output = 3 +include_trailing_comma = true +force_grid_wrap = 0 +combine_as_imports = true +order_by_type = true + +known_dask = [ + "dask", + "distributed", + "dask_cuda", +] +known_rapids = [ + "rmm", + "cudf", +] +known_first_party = [ + "dask_cudf", +] + +default_section = "THIRDPARTY" +sections = [ + "FUTURE", + "STDLIB", + "THIRDPARTY", + "DASK", + "RAPIDS", + "FIRSTPARTY", + "LOCALFOLDER", +] +skip = [ + "thirdparty", + ".eggs", + ".git", + ".hg", + ".mypy_cache", + ".tox", + ".venv", + "_build", + "buck-out", + "build", + "dist", +] diff --git a/python/dask_cudf/setup.cfg b/python/dask_cudf/setup.cfg index 66f4b8891d0..8139b3c7dc6 100644 --- a/python/dask_cudf/setup.cfg +++ b/python/dask_cudf/setup.cfg @@ -1,36 +1,5 @@ # Copyright (c) 2020-2023, NVIDIA CORPORATION. -[isort] -line_length=79 -multi_line_output=3 -include_trailing_comma=True -force_grid_wrap=0 -combine_as_imports=True -order_by_type=True -known_dask= - dask - distributed - dask_cuda -known_rapids= - rmm - cudf -known_first_party= - dask_cudf -default_section=THIRDPARTY -sections=FUTURE,STDLIB,THIRDPARTY,DASK,RAPIDS,FIRSTPARTY,LOCALFOLDER -skip= - thirdparty - .eggs - .git - .hg - .mypy_cache - .tox - .venv - _build - buck-out - build - dist - [options.entry_points] dask.dataframe.backends = cudf = dask_cudf.backends:CudfBackendEntrypoint diff --git a/setup.cfg b/setup.cfg deleted file mode 100644 index 962b7d73bbe..00000000000 --- a/setup.cfg +++ /dev/null @@ -1,64 +0,0 @@ -# Copyright (c) 2017-2023, NVIDIA CORPORATION. - -[flake8] -filename = *.py, *.pyx, *.pxd, *.pxi -exclude = __init__.py, *.egg, build, docs, .git -force-check = True -ignore = - # line break before binary operator - W503, - # whitespace before : - E203 -per-file-ignores = - # Rules ignored only in Cython: - # E211: whitespace before '(' (used in multi-line imports) - # E225: Missing whitespace around operators (breaks cython casting syntax like ) - # E226: Missing whitespace around arithmetic operators (breaks cython pointer syntax like int*) - # E227: Missing whitespace around bitwise or shift operator (Can also break casting syntax) - # E275: Missing whitespace after keyword (Doesn't work with Cython except?) - # E402: invalid syntax (works for Python, not Cython) - # E999: invalid syntax (works for Python, not Cython) - # W504: line break after binary operator (breaks lines that end with a pointer) - *.pyx: E211, E225, E226, E227, E275, E402, E999, W504 - *.pxd: E211, E225, E226, E227, E275, E402, E999, W504 - *.pxi: E211, E225, E226, E227, E275, E402, E999, W504 - -[pydocstyle] -# Due to https://github.com/PyCQA/pydocstyle/issues/363, we must exclude rather -# than include using match-dir. Note that as discussed in -# https://stackoverflow.com/questions/65478393/how-to-filter-directories-using-the-match-dir-flag-for-pydocstyle, -# unlike the match option above this match-dir will have no effect when -# pydocstyle is invoked from pre-commit. Therefore this exclusion list must -# also be maintained in the pre-commit config file. -match-dir = ^(?!(ci|cpp|conda|docs|java|notebooks)).*$ -# Allow missing docstrings for docutils -ignore-decorators = .*(docutils|doc_apply|copy_docstring).* -select = - D201, D204, D206, D207, D208, D209, D210, D211, D214, D215, D300, D301, D302, D403, D405, D406, D407, D408, D409, D410, D411, D412, D414, D418 - # Would like to enable the following rules in the future: - # D200, D202, D205, D400 - -[mypy] -ignore_missing_imports = True -# If we don't specify this, then mypy will check excluded files if -# they are imported by a checked file. -follow_imports = skip -exclude = (?x)( - cudf/_lib/ - | cudf/cudf/benchmarks/ - | cudf/cudf/tests/ - | cudf/cudf/utils/metadata/orc_column_statistics_pb2.py - | custreamz/custreamz/tests/ - | dask_cudf/dask_cudf/tests/ - # This close paren cannot be in column zero otherwise the config parser barfs - ) - -[codespell] -# note: pre-commit passes explicit lists of files here, which this skip file list doesn't override - -# this is only to allow you to run codespell interactively -skip = ./.git,./.github,./cpp/build,.*egg-info.*,./.mypy_cache,./cpp/tests,./python/cudf/cudf/tests,./java/src/test,./cpp/include/cudf_test/cxxopts.hpp -# ignore short words, and typename parameters like OffsetT -ignore-regex = \b(.{1,4}|[A-Z]\w*T)\b -ignore-words-list = inout,unparseable -builtin = clear -quiet-level = 3 From 4f2f37987fbd66de0cc9116734d2094ca4a39948 Mon Sep 17 00:00:00 2001 From: Bradley Dice Date: Fri, 24 Feb 2023 17:04:59 -0600 Subject: [PATCH 15/60] Enable nbqa pre-commit hooks for isort and black. (#12848) This enables `black` and `isort` linters for ipynb notebooks via [nbqa](https://github.com/nbQA-dev/nbQA). I propose this change to avoid manually linting notebooks like https://github.com/rapidsai/cudf/pull/12595. cc: @galipremsagar Authors: - Bradley Dice (https://github.com/bdice) Approvers: - GALI PREM SAGAR (https://github.com/galipremsagar) URL: https://github.com/rapidsai/cudf/pull/12848 --- .pre-commit-config.yaml | 10 ++ docs/cudf/source/user_guide/10min.ipynb | 1 + .../cudf/source/user_guide/cupy-interop.ipynb | 34 ++-- .../source/user_guide/guide-to-udfs.ipynb | 149 +++++++++--------- .../cudf/source/user_guide/missing-data.ipynb | 56 ++++--- 5 files changed, 141 insertions(+), 109 deletions(-) diff --git a/.pre-commit-config.yaml b/.pre-commit-config.yaml index e252af717ce..a030f3bd25b 100644 --- a/.pre-commit-config.yaml +++ b/.pre-commit-config.yaml @@ -61,6 +61,16 @@ repos: # https://github.com/PyCQA/pydocstyle/issues/603 additional_dependencies: [toml] args: ["--config=pyproject.toml"] + - repo: https://github.com/nbQA-dev/nbQA + rev: 1.6.3 + hooks: + - id: nbqa-isort + # Use the cudf_kafka isort orderings in notebooks so that dask + # and RAPIDS packages have their own sections. + args: ["--settings-file=python/cudf_kafka/pyproject.toml"] + - id: nbqa-black + # Explicitly specify the pyproject.toml at the repo root, not per-project. + args: ["--config=pyproject.toml"] - repo: https://github.com/pre-commit/mirrors-clang-format rev: v11.1.0 hooks: diff --git a/docs/cudf/source/user_guide/10min.ipynb b/docs/cudf/source/user_guide/10min.ipynb index af938b79a29..0352c624e04 100644 --- a/docs/cudf/source/user_guide/10min.ipynb +++ b/docs/cudf/source/user_guide/10min.ipynb @@ -35,6 +35,7 @@ "\n", "import cupy as cp\n", "import pandas as pd\n", + "\n", "import cudf\n", "import dask_cudf\n", "\n", diff --git a/docs/cudf/source/user_guide/cupy-interop.ipynb b/docs/cudf/source/user_guide/cupy-interop.ipynb index 3e169984ace..c98a4ddea23 100644 --- a/docs/cudf/source/user_guide/cupy-interop.ipynb +++ b/docs/cudf/source/user_guide/cupy-interop.ipynb @@ -18,9 +18,10 @@ "outputs": [], "source": [ "import timeit\n", - "from packaging import version\n", "\n", "import cupy as cp\n", + "from packaging import version\n", + "\n", "import cudf\n", "\n", "if version.parse(cp.__version__) >= version.parse(\"10.0.0\"):\n", @@ -63,10 +64,13 @@ ], "source": [ "nelem = 10000\n", - "df = cudf.DataFrame({'a':range(nelem),\n", - " 'b':range(500, nelem + 500),\n", - " 'c':range(1000, nelem + 1000)}\n", - " )\n", + "df = cudf.DataFrame(\n", + " {\n", + " \"a\": range(nelem),\n", + " \"b\": range(500, nelem + 500),\n", + " \"c\": range(1000, nelem + 1000),\n", + " }\n", + ")\n", "\n", "%timeit arr_cupy = cupy_from_dlpack(df.to_dlpack())\n", "%timeit arr_cupy = df.values\n", @@ -138,7 +142,7 @@ } ], "source": [ - "col = 'a'\n", + "col = \"a\"\n", "\n", "%timeit cola_cupy = cp.asarray(df[col])\n", "%timeit cola_cupy = cupy_from_dlpack(df[col].to_dlpack())\n", @@ -1088,14 +1092,16 @@ "metadata": {}, "outputs": [], "source": [ - "def cudf_to_cupy_sparse_matrix(data, sparseformat='column'):\n", - " \"\"\"Converts a cuDF object to a CuPy Sparse Column matrix.\n", - " \"\"\"\n", - " if sparseformat not in ('row', 'column',):\n", + "def cudf_to_cupy_sparse_matrix(data, sparseformat=\"column\"):\n", + " \"\"\"Converts a cuDF object to a CuPy Sparse Column matrix.\"\"\"\n", + " if sparseformat not in (\n", + " \"row\",\n", + " \"column\",\n", + " ):\n", " raise ValueError(\"Let's focus on column and row formats for now.\")\n", - " \n", + "\n", " _sparse_constructor = cp.sparse.csc_matrix\n", - " if sparseformat == 'row':\n", + " if sparseformat == \"row\":\n", " _sparse_constructor = cp.sparse.csr_matrix\n", "\n", " return _sparse_constructor(cupy_from_dlpack(data.to_dlpack()))" @@ -1121,8 +1127,8 @@ "nonzero = 1000\n", "for i in range(20):\n", " arr = cp.random.normal(5, 5, nelem)\n", - " arr[cp.random.choice(arr.shape[0], nelem-nonzero, replace=False)] = 0\n", - " df['a' + str(i)] = arr" + " arr[cp.random.choice(arr.shape[0], nelem - nonzero, replace=False)] = 0\n", + " df[\"a\" + str(i)] = arr" ] }, { diff --git a/docs/cudf/source/user_guide/guide-to-udfs.ipynb b/docs/cudf/source/user_guide/guide-to-udfs.ipynb index 943fc980a31..ba8c65784d2 100644 --- a/docs/cudf/source/user_guide/guide-to-udfs.ipynb +++ b/docs/cudf/source/user_guide/guide-to-udfs.ipynb @@ -15,9 +15,10 @@ "metadata": {}, "outputs": [], "source": [ + "import numpy as np\n", + "\n", "import cudf\n", - "from cudf.datasets import randomdata\n", - "import numpy as np" + "from cudf.datasets import randomdata" ] }, { @@ -375,7 +376,7 @@ "metadata": {}, "outputs": [], "source": [ - "sr = cudf.Series(['', 'abc', 'some_example'])" + "sr = cudf.Series([\"\", \"abc\", \"some_example\"])" ] }, { @@ -387,9 +388,9 @@ "source": [ "def f(st):\n", " if len(st) > 0:\n", - " if st.startswith('a'):\n", + " if st.startswith(\"a\"):\n", " return 1\n", - " elif 'example' in st:\n", + " elif \"example\" in st:\n", " return 2\n", " else:\n", " return -1\n", @@ -443,6 +444,7 @@ "outputs": [], "source": [ "from cudf.core.udf.utils import set_malloc_heap_size\n", + "\n", "set_malloc_heap_size(int(2e9))" ] }, @@ -472,7 +474,7 @@ "metadata": {}, "outputs": [], "source": [ - "df = randomdata(nrows=5, dtypes={'a':int, 'b':int, 'c':int}, seed=12)" + "df = randomdata(nrows=5, dtypes={\"a\": int, \"b\": int, \"c\": int}, seed=12)" ] }, { @@ -484,10 +486,11 @@ "source": [ "from numba import cuda\n", "\n", + "\n", "@cuda.jit\n", "def multiply(in_col, out_col, multiplier):\n", " i = cuda.grid(1)\n", - " if i < in_col.size: # boundary guard\n", + " if i < in_col.size: # boundary guard\n", " out_col[i] = in_col[i] * multiplier" ] }, @@ -508,9 +511,9 @@ "metadata": {}, "outputs": [], "source": [ - "size = len(df['a'])\n", - "df['e'] = 0.0\n", - "multiply.forall(size)(df['a'], df['e'], 10.0)" + "size = len(df[\"a\"])\n", + "df[\"e\"] = 0.0\n", + "multiply.forall(size)(df[\"a\"], df[\"e\"], 10.0)" ] }, { @@ -658,7 +661,7 @@ "outputs": [], "source": [ "def f(row):\n", - " return row['A'] + row['B']" + " return row[\"A\"] + row[\"B\"]" ] }, { @@ -733,10 +736,7 @@ } ], "source": [ - "df = cudf.DataFrame({\n", - " 'A': [1,2,3],\n", - " 'B': [4,cudf.NA,6]\n", - "})\n", + "df = cudf.DataFrame({\"A\": [1, 2, 3], \"B\": [4, cudf.NA, 6]})\n", "df" ] }, @@ -881,13 +881,14 @@ ], "source": [ "def f(row):\n", - " x = row['a']\n", + " x = row[\"a\"]\n", " if x is cudf.NA:\n", " return 0\n", " else:\n", " return x + 1\n", "\n", - "df = cudf.DataFrame({'a': [1, cudf.NA, 3]})\n", + "\n", + "df = cudf.DataFrame({\"a\": [1, cudf.NA, 3]})\n", "df" ] }, @@ -988,17 +989,15 @@ ], "source": [ "def f(row):\n", - " x = row['a']\n", - " y = row['b']\n", + " x = row[\"a\"]\n", + " y = row[\"b\"]\n", " if x + y > 3:\n", " return cudf.NA\n", " else:\n", " return x + y\n", "\n", - "df = cudf.DataFrame({\n", - " 'a': [1, 2, 3], \n", - " 'b': [2, 1, 1]\n", - "})\n", + "\n", + "df = cudf.DataFrame({\"a\": [1, 2, 3], \"b\": [2, 1, 1]})\n", "df" ] }, @@ -1099,12 +1098,10 @@ ], "source": [ "def f(row):\n", - " return row['a'] + row['b']\n", + " return row[\"a\"] + row[\"b\"]\n", + "\n", "\n", - "df = cudf.DataFrame({\n", - " 'a': [1, 2, 3], \n", - " 'b': [0.5, cudf.NA, 3.14]\n", - "})\n", + "df = cudf.DataFrame({\"a\": [1, 2, 3], \"b\": [0.5, cudf.NA, 3.14]})\n", "df" ] }, @@ -1214,15 +1211,14 @@ ], "source": [ "def f(row):\n", - " x = row['a']\n", + " x = row[\"a\"]\n", " if x > 3:\n", - " return x\n", + " return x\n", " else:\n", - " return 1.5\n", + " return 1.5\n", + "\n", "\n", - "df = cudf.DataFrame({\n", - " 'a': [1, 3, 5]\n", - "})\n", + "df = cudf.DataFrame({\"a\": [1, 3, 5]})\n", "df" ] }, @@ -1335,15 +1331,18 @@ ], "source": [ "def f(row):\n", - " return row['a'] + (row['b'] - (row['c'] / row['d'])) % row['e']\n", + " return row[\"a\"] + (row[\"b\"] - (row[\"c\"] / row[\"d\"])) % row[\"e\"]\n", "\n", - "df = cudf.DataFrame({\n", - " 'a': [1, 2, 3],\n", - " 'b': [4, 5, 6],\n", - " 'c': [cudf.NA, 4, 4],\n", - " 'd': [8, 7, 8],\n", - " 'e': [7, 1, 6]\n", - "})\n", + "\n", + "df = cudf.DataFrame(\n", + " {\n", + " \"a\": [1, 2, 3],\n", + " \"b\": [4, 5, 6],\n", + " \"c\": [cudf.NA, 4, 4],\n", + " \"d\": [8, 7, 8],\n", + " \"e\": [7, 1, 6],\n", + " }\n", + ")\n", "df" ] }, @@ -1451,10 +1450,9 @@ } ], "source": [ - "str_df = cudf.DataFrame({\n", - " 'str_col': ['abc', 'ABC', 'Example'],\n", - " 'scale': [1, 2, 3]\n", - "})\n", + "str_df = cudf.DataFrame(\n", + " {\"str_col\": [\"abc\", \"ABC\", \"Example\"], \"scale\": [1, 2, 3]}\n", + ")\n", "str_df" ] }, @@ -1466,9 +1464,9 @@ "outputs": [], "source": [ "def f(row):\n", - " st = row['str_col']\n", - " scale = row['scale']\n", - " \n", + " st = row[\"str_col\"]\n", + " scale = row[\"scale\"]\n", + "\n", " if len(st) > 5:\n", " return len(st) + scale\n", " else:\n", @@ -1626,11 +1624,12 @@ } ], "source": [ - "df = df.apply_rows(conditional_add, \n", - " incols={'a':'x', 'e':'y'},\n", - " outcols={'out': np.float64},\n", - " kwargs={}\n", - " )\n", + "df = df.apply_rows(\n", + " conditional_add,\n", + " incols={\"a\": \"x\", \"e\": \"y\"},\n", + " outcols={\"out\": np.float64},\n", + " kwargs={},\n", + ")\n", "df.head()" ] }, @@ -1738,10 +1737,11 @@ " for i, (x, y) in enumerate(zip(a, b)):\n", " out[i] = x + y\n", "\n", - "df = randomdata(nrows=5, dtypes={'a':int, 'b':int, 'c':int}, seed=12)\n", - "df.loc[2, 'a'] = None\n", - "df.loc[3, 'b'] = None\n", - "df.loc[1, 'c'] = None\n", + "\n", + "df = randomdata(nrows=5, dtypes={\"a\": int, \"b\": int, \"c\": int}, seed=12)\n", + "df.loc[2, \"a\"] = None\n", + "df.loc[3, \"b\"] = None\n", + "df.loc[1, \"c\"] = None\n", "df.head()" ] }, @@ -1841,10 +1841,9 @@ } ], "source": [ - "df = df.apply_rows(gpu_add, \n", - " incols=['a', 'b'],\n", - " outcols={'out':np.float64},\n", - " kwargs={})\n", + "df = df.apply_rows(\n", + " gpu_add, incols=[\"a\", \"b\"], outcols={\"out\": np.float64}, kwargs={}\n", + ")\n", "df.head()" ] }, @@ -1892,7 +1891,7 @@ } ], "source": [ - "ser = cudf.Series([16, 25, 36, 49, 64, 81], dtype='float64')\n", + "ser = cudf.Series([16, 25, 36, 49, 64, 81], dtype=\"float64\")\n", "ser" ] }, @@ -1935,12 +1934,13 @@ "source": [ "import math\n", "\n", + "\n", "def example_func(window):\n", " b = 0\n", " for a in window:\n", " b = max(b, math.sqrt(a))\n", " if b == 8:\n", - " return 100 \n", + " return 100\n", " return b" ] }, @@ -2064,8 +2064,8 @@ ], "source": [ "df2 = cudf.DataFrame()\n", - "df2['a'] = np.arange(55, 65, dtype='float64')\n", - "df2['b'] = np.arange(55, 65, dtype='float64')\n", + "df2[\"a\"] = np.arange(55, 65, dtype=\"float64\")\n", + "df2[\"b\"] = np.arange(55, 65, dtype=\"float64\")\n", "df2.head()" ] }, @@ -2279,7 +2279,9 @@ } ], "source": [ - "df = randomdata(nrows=10, dtypes={'a':float, 'b':bool, 'c':str, 'e': float}, seed=12)\n", + "df = randomdata(\n", + " nrows=10, dtypes={\"a\": float, \"b\": bool, \"c\": str, \"e\": float}, seed=12\n", + ")\n", "df.head()" ] }, @@ -2290,7 +2292,7 @@ "metadata": {}, "outputs": [], "source": [ - "grouped = df.groupby(['b'])" + "grouped = df.groupby([\"b\"])" ] }, { @@ -2469,9 +2471,9 @@ } ], "source": [ - "results = grouped.apply_grouped(rolling_avg,\n", - " incols=['e'],\n", - " outcols=dict(rolling_avg_e=np.float64))\n", + "results = grouped.apply_grouped(\n", + " rolling_avg, incols=[\"e\"], outcols=dict(rolling_avg_e=np.float64)\n", + ")\n", "results" ] }, @@ -2554,8 +2556,9 @@ " i = cuda.grid(1)\n", " if i < x.size:\n", " out[i] = x[i] * 5\n", - " \n", - "out = cudf.Series(cp.zeros(len(s), dtype='int32'))\n", + "\n", + "\n", + "out = cudf.Series(cp.zeros(len(s), dtype=\"int32\"))\n", "multiply_by_5.forall(s.shape[0])(s, out)\n", "out" ] diff --git a/docs/cudf/source/user_guide/missing-data.ipynb b/docs/cudf/source/user_guide/missing-data.ipynb index ac5bddd34cf..f1404ce0b77 100644 --- a/docs/cudf/source/user_guide/missing-data.ipynb +++ b/docs/cudf/source/user_guide/missing-data.ipynb @@ -39,8 +39,9 @@ "metadata": {}, "outputs": [], "source": [ - "import cudf\n", - "import numpy as np" + "import numpy as np\n", + "\n", + "import cudf" ] }, { @@ -50,7 +51,7 @@ "metadata": {}, "outputs": [], "source": [ - "df = cudf.DataFrame({'a': [1, 2, None, 4], 'b':[0.1, None, 2.3, 17.17]})" + "df = cudf.DataFrame({\"a\": [1, 2, None, 4], \"b\": [0.1, None, 2.3, 17.17]})" ] }, { @@ -221,7 +222,7 @@ } ], "source": [ - "df['a'].notna()" + "df[\"a\"].notna()" ] }, { @@ -304,7 +305,7 @@ } ], "source": [ - "df['b'] == np.nan" + "df[\"b\"] == np.nan" ] }, { @@ -535,7 +536,10 @@ ], "source": [ "import pandas as pd\n", - "datetime_series = cudf.Series([pd.Timestamp(\"20120101\"), pd.NaT, pd.Timestamp(\"20120101\")])\n", + "\n", + "datetime_series = cudf.Series(\n", + " [pd.Timestamp(\"20120101\"), pd.NaT, pd.Timestamp(\"20120101\")]\n", + ")\n", "datetime_series" ] }, @@ -618,7 +622,12 @@ "metadata": {}, "outputs": [], "source": [ - "df1 = cudf.DataFrame({'a':[1, None, 2, 3, None], 'b':cudf.Series([np.nan, 2, 3.2, 0.1, 1], nan_as_null=False)})" + "df1 = cudf.DataFrame(\n", + " {\n", + " \"a\": [1, None, 2, 3, None],\n", + " \"b\": cudf.Series([np.nan, 2, 3.2, 0.1, 1], nan_as_null=False),\n", + " }\n", + ")" ] }, { @@ -628,7 +637,9 @@ "metadata": {}, "outputs": [], "source": [ - "df2 = cudf.DataFrame({'a':[1, 11, 2, 34, 10], 'b':cudf.Series([0.23, 22, 3.2, None, 1])})" + "df2 = cudf.DataFrame(\n", + " {\"a\": [1, 11, 2, 34, 10], \"b\": cudf.Series([0.23, 22, 3.2, None, 1])}\n", + ")" ] }, { @@ -899,7 +910,7 @@ } ], "source": [ - "df1['a']" + "df1[\"a\"]" ] }, { @@ -920,7 +931,7 @@ } ], "source": [ - "df1['a'].sum()" + "df1[\"a\"].sum()" ] }, { @@ -949,7 +960,7 @@ } ], "source": [ - "df1['a'].mean()" + "df1[\"a\"].mean()" ] }, { @@ -980,7 +991,7 @@ } ], "source": [ - "df1['a'].sum(skipna=False)" + "df1[\"a\"].sum(skipna=False)" ] }, { @@ -1001,7 +1012,7 @@ } ], "source": [ - "df1['a'].mean(skipna=False)" + "df1[\"a\"].mean(skipna=False)" ] }, { @@ -1035,7 +1046,7 @@ } ], "source": [ - "df1['a'].cumsum()" + "df1[\"a\"].cumsum()" ] }, { @@ -1069,7 +1080,7 @@ } ], "source": [ - "df1['a'].cumsum(skipna=False)" + "df1[\"a\"].cumsum(skipna=False)" ] }, { @@ -1148,7 +1159,7 @@ } ], "source": [ - "cudf.Series([], dtype='float64').sum()" + "cudf.Series([], dtype=\"float64\").sum()" ] }, { @@ -1219,7 +1230,7 @@ } ], "source": [ - "cudf.Series([], dtype='float64').prod()" + "cudf.Series([], dtype=\"float64\").prod()" ] }, { @@ -1382,7 +1393,7 @@ } ], "source": [ - "df1.groupby('a').mean()" + "df1.groupby(\"a\").mean()" ] }, { @@ -1463,7 +1474,7 @@ } ], "source": [ - "df1.groupby('a', dropna=False).mean()" + "df1.groupby(\"a\", dropna=False).mean()" ] }, { @@ -1670,7 +1681,7 @@ } ], "source": [ - "df1['b'].fillna(10)" + "df1[\"b\"].fillna(10)" ] }, { @@ -1697,7 +1708,8 @@ "outputs": [], "source": [ "import cupy as cp\n", - "dff = cudf.DataFrame(cp.random.randn(10, 3), columns=list('ABC'))" + "\n", + "dff = cudf.DataFrame(cp.random.randn(10, 3), columns=list(\"ABC\"))" ] }, { @@ -2339,7 +2351,7 @@ } ], "source": [ - "df1['a'].dropna()" + "df1[\"a\"].dropna()" ] }, { From d14d980b63402a779a3f75cc64cb3a5a0be7898d Mon Sep 17 00:00:00 2001 From: Vyas Ramasubramani Date: Fri, 24 Feb 2023 15:30:07 -0800 Subject: [PATCH 16/60] Add dfg as a pre-commit hook (#12819) This change allows local and remote runs to handle calls to dfg identically, and removes the need for a separate CI check. Authors: - Vyas Ramasubramani (https://github.com/vyasr) Approvers: - AJ Schmidt (https://github.com/ajschmidt8) URL: https://github.com/rapidsai/cudf/pull/12819 --- .github/workflows/pr.yaml | 2 ++ .pre-commit-config.yaml | 5 +++++ 2 files changed, 7 insertions(+) diff --git a/.github/workflows/pr.yaml b/.github/workflows/pr.yaml index 952b58abda5..3a80139e333 100644 --- a/.github/workflows/pr.yaml +++ b/.github/workflows/pr.yaml @@ -30,6 +30,8 @@ jobs: checks: secrets: inherit uses: rapidsai/shared-action-workflows/.github/workflows/checks.yaml@branch-23.04 + with: + enable_check_generated_files: false conda-cpp-build: needs: checks secrets: inherit diff --git a/.pre-commit-config.yaml b/.pre-commit-config.yaml index a030f3bd25b..1eb2c508db9 100644 --- a/.pre-commit-config.yaml +++ b/.pre-commit-config.yaml @@ -160,6 +160,11 @@ repos: .*test.*| ^CHANGELOG.md$ ) + - repo: https://github.com/rapidsai/dependency-file-generator + rev: v1.4.0 + hooks: + - id: rapids-dependency-file-generator + args: ["--clean"] default_language_version: python: python3 From eb4da9345f172c3911f78c5e851757ec2ec222b9 Mon Sep 17 00:00:00 2001 From: Carl Simon Adorf Date: Sat, 25 Feb 2023 01:13:34 +0100 Subject: [PATCH 17/60] CI: Remove specification of manual stage for check_style.sh script. (#12803) Do not explicitly specify to run the "manual" stage when running pre-commits as part of the ci/check_style.sh script. Authors: - Carl Simon Adorf (https://github.com/csadorf) - Vyas Ramasubramani (https://github.com/vyasr) - Bradley Dice (https://github.com/bdice) Approvers: - Vyas Ramasubramani (https://github.com/vyasr) - AJ Schmidt (https://github.com/ajschmidt8) URL: https://github.com/rapidsai/cudf/pull/12803 --- ci/check_style.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/ci/check_style.sh b/ci/check_style.sh index 020143095ce..f9bfea7b47c 100755 --- a/ci/check_style.sh +++ b/ci/check_style.sh @@ -1,5 +1,5 @@ #!/bin/bash -# Copyright (c) 2020-2022, NVIDIA CORPORATION. +# Copyright (c) 2020-2023, NVIDIA CORPORATION. set -euo pipefail @@ -20,4 +20,4 @@ mkdir -p $(dirname ${RAPIDS_CMAKE_FORMAT_FILE}) wget -O ${RAPIDS_CMAKE_FORMAT_FILE} ${FORMAT_FILE_URL} # Run pre-commit checks -pre-commit run --hook-stage manual --all-files --show-diff-on-failure +pre-commit run --all-files --show-diff-on-failure From 173459e99fa8fce4202e2613f1f2c89a016cd350 Mon Sep 17 00:00:00 2001 From: Vyas Ramasubramani Date: Fri, 24 Feb 2023 21:12:33 -0800 Subject: [PATCH 18/60] Replace message parsing with throwing more specific exceptions (#12426) This PR identifies places where cuDF Python is parsing exception messages from libcudf in order to throw an appropriate Python exception and modifies the associated libcudf error macros (`CUDF_EXPECTS` or `CUDF_FAIL`) to throw a more descriptive exception. In order to fully support this behavior with arbitrary libcudf exceptions, this PR also introduces a custom Cython exception handler that can be extended to catch and handle any new type of exception thrown by libcudf. To cases where issues with the dtype of an argument is treated as a TypeError rather than a ValueError in pandas, a new exception type `dtype_error : public logic_error` is defined and mapped to a TypeError in Python. This PR does not aim to exhaustively improve the choice of thrown exceptions throughout libcudf, only in places that are immediately relevant to removing message parsing in cudf Python. Resolves #10632 Authors: - Vyas Ramasubramani (https://github.com/vyasr) Approvers: - Robert (Bobby) Evans (https://github.com/revans2) - Ashwin Srinath (https://github.com/shwina) - Matthew Roeschke (https://github.com/mroeschke) - David Wendt (https://github.com/davidwendt) - Nghia Truong (https://github.com/ttnghia) URL: https://github.com/rapidsai/cudf/pull/12426 --- cpp/include/cudf/binaryop.hpp | 5 +- cpp/include/cudf/concatenate.hpp | 10 +-- cpp/include/cudf/lists/combine.hpp | 8 +- cpp/include/cudf/lists/contains.hpp | 9 +-- cpp/include/cudf/lists/gather.hpp | 4 +- cpp/include/cudf/strings/json.hpp | 4 +- cpp/include/cudf/utilities/error.hpp | 25 +++++- cpp/src/binaryop/binaryop.cpp | 2 +- cpp/src/binaryop/compiled/equality_ops.cu | 5 +- .../binaryop/compiled/struct_binary_ops.cuh | 3 +- cpp/src/copying/concatenate.cu | 11 ++- .../combine/concatenate_list_elements.cu | 8 +- cpp/src/lists/contains.cu | 3 +- cpp/src/lists/copying/segmented_gather.cu | 4 +- cpp/src/strings/json/json_path.cu | 7 +- .../binop-compiled-fixed_point-test.cpp | 2 +- cpp/tests/copying/concatenate_tests.cu | 25 +++--- .../copying/segmented_gather_list_tests.cpp | 6 +- .../concatenate_list_elements_tests.cpp | 8 +- cpp/tests/lists/contains_tests.cpp | 33 +++++--- cpp/tests/strings/json_tests.cpp | 10 ++- .../developer_guide/contributing_guide.md | 16 ++-- java/src/main/native/include/jni_utils.hpp | 6 +- python/cudf/cudf/_lib/CMakeLists.txt | 5 ++ python/cudf/cudf/_lib/cpp/copying.pxd | 5 +- python/cudf/cudf/_lib/cpp/lists/contains.pxd | 9 ++- python/cudf/cudf/_lib/exception_handler.hpp | 80 +++++++++++++++++++ python/cudf/cudf/_lib/exception_handler.pxd | 5 ++ python/cudf/cudf/core/column/column.py | 17 +--- python/cudf/cudf/core/column/decimal.py | 48 +++++------ python/cudf/cudf/core/column/lists.py | 67 ++++------------ python/cudf/cudf/core/column/string.py | 33 +++----- python/cudf/cudf/tests/test_column.py | 5 +- 33 files changed, 284 insertions(+), 204 deletions(-) create mode 100644 python/cudf/cudf/_lib/exception_handler.hpp create mode 100644 python/cudf/cudf/_lib/exception_handler.pxd diff --git a/cpp/include/cudf/binaryop.hpp b/cpp/include/cudf/binaryop.hpp index 6371cb6c82b..77d6a4d1e89 100644 --- a/cpp/include/cudf/binaryop.hpp +++ b/cpp/include/cudf/binaryop.hpp @@ -1,5 +1,5 @@ /* - * Copyright (c) 2019-2022, NVIDIA CORPORATION. + * Copyright (c) 2019-2023, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -108,6 +108,7 @@ enum class binary_operator : int32_t { * @throw cudf::logic_error if @p output_type dtype isn't fixed-width * @throw cudf::logic_error if @p output_type dtype isn't boolean for comparison and logical * operations. + * @throw cudf::data_type_error if the operation is not supported for the types of @p lhs and @p rhs */ std::unique_ptr binary_operation( scalar const& lhs, @@ -136,6 +137,7 @@ std::unique_ptr binary_operation( * @throw cudf::logic_error if @p output_type dtype isn't fixed-width * @throw cudf::logic_error if @p output_type dtype isn't boolean for comparison and logical * operations. + * @throw cudf::data_type_error if the operation is not supported for the types of @p lhs and @p rhs */ std::unique_ptr binary_operation( column_view const& lhs, @@ -163,6 +165,7 @@ std::unique_ptr binary_operation( * @throw cudf::logic_error if @p output_type dtype isn't boolean for comparison and logical * operations. * @throw cudf::logic_error if @p output_type dtype isn't fixed-width + * @throw cudf::data_type_error if the operation is not supported for the types of @p lhs and @p rhs */ std::unique_ptr binary_operation( column_view const& lhs, diff --git a/cpp/include/cudf/concatenate.hpp b/cpp/include/cudf/concatenate.hpp index 1f8ce65ad93..b20c97b3c31 100644 --- a/cpp/include/cudf/concatenate.hpp +++ b/cpp/include/cudf/concatenate.hpp @@ -1,5 +1,5 @@ /* - * Copyright (c) 2020-2022, NVIDIA CORPORATION. + * Copyright (c) 2020-2023, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -50,8 +50,8 @@ rmm::device_buffer concatenate_masks( /** * @brief Concatenates multiple columns into a single column. * - * @throws cudf::logic_error - * If types of the input columns mismatch + * @throws cudf::logic_error If types of the input columns mismatch + * @throws std::overflow_error If the the total number of output rows exceeds cudf::size_type * * @param columns_to_concat host_span of column views to be concatenated into a single column * @param mr Device memory resource used to allocate the returned column's device memory @@ -80,8 +80,8 @@ std::unique_ptr concatenate( * column_view tc1 = (t->view()).column(1); //Contains {0,1,2,3,4,5,6,7} * ``` * - * @throws cudf::logic_error - * If number of columns mismatch + * @throws cudf::logic_error If number of columns mismatch + * @throws std::overflow_error If the the total number of output rows exceeds cudf::size_type * * @param tables_to_concat host_span of table views to be concatenated into a single table * @param mr Device memory resource used to allocate the returned table's device memory diff --git a/cpp/include/cudf/lists/combine.hpp b/cpp/include/cudf/lists/combine.hpp index 4f211e87cc7..531396e940e 100644 --- a/cpp/include/cudf/lists/combine.hpp +++ b/cpp/include/cudf/lists/combine.hpp @@ -1,5 +1,5 @@ /* - * Copyright (c) 2021-2022, NVIDIA CORPORATION. + * Copyright (c) 2021-2023, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -80,10 +80,10 @@ std::unique_ptr concatenate_rows( * r is [ {1, 2, 3, 4, 5}, {6, 7, 8, 9} ] * @endcode * - * @throws cudf::logic_error if the input column is not at least two-level depth lists column (i.e., - * each row must be a list of list). + * @throws std::invalid_argument if the input column is not at least two-level depth lists column + * (i.e., each row must be a list of list). * @throws cudf::logic_error if the input lists column contains nested typed entries that are not - * lists. + * lists. * * @param input The lists column containing lists of list elements to concatenate. * @param null_policy The parameter to specify whether a null list element will be ignored from diff --git a/cpp/include/cudf/lists/contains.hpp b/cpp/include/cudf/lists/contains.hpp index d2b4d59dfba..a9f06bf399c 100644 --- a/cpp/include/cudf/lists/contains.hpp +++ b/cpp/include/cudf/lists/contains.hpp @@ -1,5 +1,5 @@ /* - * Copyright (c) 2021-2022, NVIDIA CORPORATION. + * Copyright (c) 2021-2023, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -126,9 +126,7 @@ enum class duplicate_find_option : int32_t { * @param mr Device memory resource used to allocate the returned column's device memory. * @return std::unique_ptr INT32 column of `n` rows with the location of the `search_key` * - * @throw cudf::logic_error If `search_key` type does not match the element type in `lists` - * @throw cudf::logic_error If `search_key` is of a nested type, or `lists` contains nested - * elements (LIST, STRUCT) + * @throw cudf::data_type_error If `search_keys` type does not match the element type in `lists` */ std::unique_ptr index_of( cudf::lists_column_view const& lists, @@ -163,8 +161,7 @@ std::unique_ptr index_of( * @return std::unique_ptr INT32 column of `n` rows with the location of the `search_key` * * @throw cudf::logic_error If `search_keys` does not match `lists` in its number of rows - * @throw cudf::logic_error If `search_keys` type does not match the element type in `lists` - * @throw cudf::logic_error If `lists` or `search_keys` contains nested elements (LIST, STRUCT) + * @throw cudf::data_type_error If `search_keys` type does not match the element type in `lists` */ std::unique_ptr index_of( cudf::lists_column_view const& lists, diff --git a/cpp/include/cudf/lists/gather.hpp b/cpp/include/cudf/lists/gather.hpp index f91ce29a7cb..38bed9ede43 100644 --- a/cpp/include/cudf/lists/gather.hpp +++ b/cpp/include/cudf/lists/gather.hpp @@ -1,5 +1,5 @@ /* - * Copyright (c) 2021-2022, NVIDIA CORPORATION. + * Copyright (c) 2021-2023, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -43,7 +43,7 @@ namespace lists { * @endcode * * @throws cudf::logic_error if `gather_map_list` size is not same as `source_column` size. - * @throws cudf::logic_error if gather_map contains null values. + * @throws std::invalid_argument if gather_map contains null values. * @throws cudf::logic_error if gather_map is not list column of an index type. * * If indices in `gather_map_list` are outside the range `[-n, n)`, where `n` is the number of diff --git a/cpp/include/cudf/strings/json.hpp b/cpp/include/cudf/strings/json.hpp index 11e8daa9855..8fabee6b9a5 100644 --- a/cpp/include/cudf/strings/json.hpp +++ b/cpp/include/cudf/strings/json.hpp @@ -1,5 +1,5 @@ /* - * Copyright (c) 2021-2022, NVIDIA CORPORATION. + * Copyright (c) 2021-2023, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -160,6 +160,8 @@ class get_json_object_options { * @param options Options for controlling the behavior of the function * @param mr Resource for allocating device memory. * @return New strings column containing the retrieved json object strings + * + * @throw std::invalid_argument if provided an invalid operator or an empty name */ std::unique_ptr get_json_object( cudf::strings_column_view const& col, diff --git a/cpp/include/cudf/utilities/error.hpp b/cpp/include/cudf/utilities/error.hpp index 38ca0f2651e..f70ef4e5f07 100644 --- a/cpp/include/cudf/utilities/error.hpp +++ b/cpp/include/cudf/utilities/error.hpp @@ -1,5 +1,5 @@ /* - * Copyright (c) 2019-2022, NVIDIA CORPORATION. + * Copyright (c) 2019-2023, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -84,6 +84,29 @@ struct cuda_error : public std::runtime_error { struct fatal_cuda_error : public cuda_error { using cuda_error::cuda_error; // Inherit constructors }; + +/** + * @brief Exception thrown when an operation is attempted on an unsupported dtype. + * + * This exception should be thrown when an operation is attempted on an + * unsupported data_type. This exception should not be thrown directly and is + * instead thrown by the CUDF_EXPECTS or CUDF_FAIL macros. + */ +struct data_type_error : public std::invalid_argument { + /** + * @brief Constructs a data_type_error with the error message. + * + * @param message Message to be associated with the exception + */ + data_type_error(char const* const message) : std::invalid_argument(message) {} + + /** + * @brief Construct a new data_type_error object with error message + * + * @param message Message to be associated with the exception + */ + data_type_error(std::string const& message) : std::invalid_argument(message) {} +}; /** @} */ } // namespace cudf diff --git a/cpp/src/binaryop/binaryop.cpp b/cpp/src/binaryop/binaryop.cpp index b23c1fc9fe1..f81f0dcc311 100644 --- a/cpp/src/binaryop/binaryop.cpp +++ b/cpp/src/binaryop/binaryop.cpp @@ -203,7 +203,7 @@ std::unique_ptr binary_operation(LhsType const& lhs, return cudf::binops::compiled::string_null_min_max(lhs, rhs, op, output_type, stream, mr); if (not cudf::binops::compiled::is_supported_operation(output_type, lhs.type(), rhs.type(), op)) - CUDF_FAIL("Unsupported operator for these types"); + CUDF_FAIL("Unsupported operator for these types", cudf::data_type_error); if (cudf::is_fixed_point(lhs.type()) or cudf::is_fixed_point(rhs.type())) { cudf::binops::compiled::fixed_point_binary_operation_validation( diff --git a/cpp/src/binaryop/compiled/equality_ops.cu b/cpp/src/binaryop/compiled/equality_ops.cu index 61f02252a26..041fca76494 100644 --- a/cpp/src/binaryop/compiled/equality_ops.cu +++ b/cpp/src/binaryop/compiled/equality_ops.cu @@ -1,5 +1,5 @@ /* - * Copyright (c) 2021-2022, NVIDIA CORPORATION. + * Copyright (c) 2021-2023, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -26,7 +26,8 @@ void dispatch_equality_op(mutable_column_view& out, rmm::cuda_stream_view stream) { CUDF_EXPECTS(op == binary_operator::EQUAL || op == binary_operator::NOT_EQUAL, - "Unsupported operator for these types"); + "Unsupported operator for these types", + cudf::data_type_error); auto common_dtype = get_common_type(out.type(), lhs.type(), rhs.type()); auto outd = mutable_column_device_view::create(out, stream); auto lhsd = column_device_view::create(lhs, stream); diff --git a/cpp/src/binaryop/compiled/struct_binary_ops.cuh b/cpp/src/binaryop/compiled/struct_binary_ops.cuh index d167f0fe3c5..8418493318f 100644 --- a/cpp/src/binaryop/compiled/struct_binary_ops.cuh +++ b/cpp/src/binaryop/compiled/struct_binary_ops.cuh @@ -149,7 +149,8 @@ void apply_struct_equality_op(mutable_column_view& out, { CUDF_EXPECTS(op == binary_operator::EQUAL || op == binary_operator::NOT_EQUAL || op == binary_operator::NULL_EQUALS, - "Unsupported operator for these types"); + "Unsupported operator for these types", + cudf::data_type_error); auto tlhs = table_view{{lhs}}; auto trhs = table_view{{rhs}}; diff --git a/cpp/src/copying/concatenate.cu b/cpp/src/copying/concatenate.cu index 577d6427b19..5d36d70696c 100644 --- a/cpp/src/copying/concatenate.cu +++ b/cpp/src/copying/concatenate.cu @@ -1,5 +1,5 @@ /* - * Copyright (c) 2020-2022, NVIDIA CORPORATION. + * Copyright (c) 2020-2023, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -228,7 +228,8 @@ std::unique_ptr fused_concatenate(host_span views, auto const output_size = std::get<3>(device_views); CUDF_EXPECTS(output_size <= static_cast(std::numeric_limits::max()), - "Total number of concatenated rows exceeds size_type range"); + "Total number of concatenated rows exceeds size_type range", + std::overflow_error); // Allocate output auto const policy = has_nulls ? mask_policy::ALWAYS : mask_policy::NEVER; @@ -398,7 +399,8 @@ void traverse_children::operator()(host_span(std::numeric_limits::max()), - "Total number of concatenated chars exceeds size_type range"); + "Total number of concatenated chars exceeds size_type range", + std::overflow_error); } template <> @@ -469,7 +471,8 @@ void bounds_and_type_check(host_span cols, rmm::cuda_stream_v }); // note: output text must include "exceeds size_type range" for python error handling CUDF_EXPECTS(total_row_count <= static_cast(std::numeric_limits::max()), - "Total number of concatenated rows exceeds size_type range"); + "Total number of concatenated rows exceeds size_type range", + std::overflow_error); // traverse children cudf::type_dispatcher(cols.front().type(), traverse_children{}, cols, stream); diff --git a/cpp/src/lists/combine/concatenate_list_elements.cu b/cpp/src/lists/combine/concatenate_list_elements.cu index 496d9ee670a..257b0aed82f 100644 --- a/cpp/src/lists/combine/concatenate_list_elements.cu +++ b/cpp/src/lists/combine/concatenate_list_elements.cu @@ -1,5 +1,5 @@ /* - * Copyright (c) 2021-2022, NVIDIA CORPORATION. + * Copyright (c) 2021-2023, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -257,11 +257,13 @@ std::unique_ptr concatenate_list_elements(column_view const& input, rmm::mr::device_memory_resource* mr) { auto type = input.type(); // Column that is lists of lists. - CUDF_EXPECTS(type.id() == type_id::LIST, "Input column must be a lists column."); + CUDF_EXPECTS( + type.id() == type_id::LIST, "Input column must be a lists column.", std::invalid_argument); auto col = lists_column_view(input).child(); // Rows, which are lists. type = col.type(); - CUDF_EXPECTS(type.id() == type_id::LIST, "Rows of the input column must be lists."); + CUDF_EXPECTS( + type.id() == type_id::LIST, "Rows of the input column must be lists.", std::invalid_argument); col = lists_column_view(col).child(); // The last level entries what we need to check. type = col.type(); diff --git a/cpp/src/lists/contains.cu b/cpp/src/lists/contains.cu index 05fe82d1713..a3293e36825 100644 --- a/cpp/src/lists/contains.cu +++ b/cpp/src/lists/contains.cu @@ -309,7 +309,8 @@ struct dispatch_index_of { auto const child = lists.child(); CUDF_EXPECTS(child.type() == search_keys.type(), - "Type/Scale of search key does not match list column element type."); + "Type/Scale of search key does not match list column element type.", + cudf::data_type_error); CUDF_EXPECTS(search_keys.type().id() != type_id::EMPTY, "Type cannot be empty."); auto constexpr search_key_is_scalar = std::is_same_v; diff --git a/cpp/src/lists/copying/segmented_gather.cu b/cpp/src/lists/copying/segmented_gather.cu index 2c12e09bcd9..79d33e7c17d 100644 --- a/cpp/src/lists/copying/segmented_gather.cu +++ b/cpp/src/lists/copying/segmented_gather.cu @@ -1,5 +1,5 @@ /* - * Copyright (c) 2020-2022, NVIDIA CORPORATION. + * Copyright (c) 2020-2023, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -38,7 +38,7 @@ std::unique_ptr segmented_gather(lists_column_view const& value_column, { CUDF_EXPECTS(is_index_type(gather_map.child().type()), "Gather map should be list column of index type"); - CUDF_EXPECTS(!gather_map.has_nulls(), "Gather map contains nulls"); + CUDF_EXPECTS(!gather_map.has_nulls(), "Gather map contains nulls", std::invalid_argument); CUDF_EXPECTS(value_column.size() == gather_map.size(), "Gather map and list column should be same size"); diff --git a/cpp/src/strings/json/json_path.cu b/cpp/src/strings/json/json_path.cu index afe16518036..c6ea47ec0f3 100644 --- a/cpp/src/strings/json/json_path.cu +++ b/cpp/src/strings/json/json_path.cu @@ -1,5 +1,5 @@ /* - * Copyright (c) 2021-2022, NVIDIA CORPORATION. + * Copyright (c) 2021-2023, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -588,7 +588,7 @@ class path_state : private parser { return path_operator{path_operator_type::CHILD_WILDCARD}; } break; - default: CUDF_FAIL("Unrecognized JSONPath operator"); break; + default: CUDF_FAIL("Unrecognized JSONPath operator", std::invalid_argument); break; } return {path_operator_type::ERROR}; } @@ -624,7 +624,8 @@ class path_state : private parser { } // an empty name is not valid - CUDF_EXPECTS(name.size_bytes() > 0, "Invalid empty name in JSONPath query string"); + CUDF_EXPECTS( + name.size_bytes() > 0, "Invalid empty name in JSONPath query string", std::invalid_argument); return true; } diff --git a/cpp/tests/binaryop/binop-compiled-fixed_point-test.cpp b/cpp/tests/binaryop/binop-compiled-fixed_point-test.cpp index 8170fe4f490..bdd0003b86b 100644 --- a/cpp/tests/binaryop/binop-compiled-fixed_point-test.cpp +++ b/cpp/tests/binaryop/binop-compiled-fixed_point-test.cpp @@ -678,7 +678,7 @@ TYPED_TEST(FixedPointCompiledTest, FixedPointBinaryOpThrows) auto const col = fp_wrapper{{100, 300, 500, 700}, scale_type{-2}}; auto const non_bool_type = cudf::data_type{cudf::type_to_id(), -2}; EXPECT_THROW(cudf::binary_operation(col, col, cudf::binary_operator::LESS, non_bool_type), - cudf::logic_error); + cudf::data_type_error); } TYPED_TEST(FixedPointCompiledTest, FixedPointBinaryOpModSimple) diff --git a/cpp/tests/copying/concatenate_tests.cu b/cpp/tests/copying/concatenate_tests.cu index 79ec2293455..ca343b963d7 100644 --- a/cpp/tests/copying/concatenate_tests.cu +++ b/cpp/tests/copying/concatenate_tests.cu @@ -36,6 +36,7 @@ #include #include +#include #include template @@ -369,7 +370,7 @@ TEST_F(OverflowTest, OverflowTest) cudf::table_view tbl({*many_chars}); EXPECT_THROW(cudf::concatenate(std::vector({tbl, tbl, tbl, tbl, tbl, tbl})), - cudf::logic_error); + std::overflow_error); } // string column, overflow on chars @@ -384,7 +385,7 @@ TEST_F(OverflowTest, OverflowTest) cudf::table_view tbl({*col}); EXPECT_THROW(cudf::concatenate(std::vector({tbl, tbl, tbl, tbl, tbl, tbl})), - cudf::logic_error); + std::overflow_error); } // string column, overflow on offsets (rows) @@ -400,7 +401,7 @@ TEST_F(OverflowTest, OverflowTest) cudf::table_view tbl({*col}); EXPECT_THROW(cudf::concatenate(std::vector({tbl, tbl, tbl, tbl, tbl, tbl})), - cudf::logic_error); + std::overflow_error); } // list, structs too long @@ -425,7 +426,7 @@ TEST_F(OverflowTest, OverflowTest) cudf::table_view tbl({*col}); auto tables = std::vector({tbl, tbl, tbl, tbl, tbl, tbl, tbl, tbl, tbl, tbl, tbl, tbl}); - EXPECT_THROW(cudf::concatenate(tables), cudf::logic_error); + EXPECT_THROW(cudf::concatenate(tables), std::overflow_error); } // struct, list child too long @@ -450,7 +451,7 @@ TEST_F(OverflowTest, OverflowTest) cudf::table_view tbl({*col}); auto tables = std::vector({tbl, tbl, tbl, tbl, tbl, tbl, tbl, tbl, tbl, tbl, tbl, tbl}); - EXPECT_THROW(cudf::concatenate(tables), cudf::logic_error); + EXPECT_THROW(cudf::concatenate(tables), std::overflow_error); } } @@ -470,7 +471,8 @@ TEST_F(OverflowTest, Presliced) // 513 * 1024 * 1024, should fail cudf::table_view b({sliced[1]}); - EXPECT_THROW(cudf::concatenate(std::vector({b, b, b, b})), cudf::logic_error); + EXPECT_THROW(cudf::concatenate(std::vector({b, b, b, b})), + std::overflow_error); } // struct column @@ -490,7 +492,8 @@ TEST_F(OverflowTest, Presliced) // 513 * 1024 * 1024, should fail cudf::table_view b({sliced[1]}); - EXPECT_THROW(cudf::concatenate(std::vector({b, b, b, b})), cudf::logic_error); + EXPECT_THROW(cudf::concatenate(std::vector({b, b, b, b})), + std::overflow_error); } // strings, overflow on chars @@ -516,7 +519,8 @@ TEST_F(OverflowTest, Presliced) // (num_rows / 2) + 1 should fail cudf::table_view b({sliced[1]}); - EXPECT_THROW(cudf::concatenate(std::vector({b, b, b, b})), cudf::logic_error); + EXPECT_THROW(cudf::concatenate(std::vector({b, b, b, b})), + std::overflow_error); } // strings, overflow on offsets @@ -589,7 +593,7 @@ TEST_F(OverflowTest, Presliced) auto sliced = cudf::split(*col, {2}); cudf::table_view tbl({sliced[1]}); auto tables = std::vector({tbl, tbl, tbl, tbl}); - EXPECT_THROW(cudf::concatenate(tables), cudf::logic_error); + EXPECT_THROW(cudf::concatenate(tables), std::overflow_error); } // list, overflow on offsets @@ -674,7 +678,8 @@ TEST_F(OverflowTest, Presliced) cudf::concatenate(std::vector({a, a, a, a})); cudf::table_view b({sliced[1]}); - EXPECT_THROW(cudf::concatenate(std::vector({b, b, b, b})), cudf::logic_error); + EXPECT_THROW(cudf::concatenate(std::vector({b, b, b, b})), + std::overflow_error); } } diff --git a/cpp/tests/copying/segmented_gather_list_tests.cpp b/cpp/tests/copying/segmented_gather_list_tests.cpp index deeebc641c2..fc21af2087b 100644 --- a/cpp/tests/copying/segmented_gather_list_tests.cpp +++ b/cpp/tests/copying/segmented_gather_list_tests.cpp @@ -1,5 +1,5 @@ /* - * Copyright (c) 2020-2022, NVIDIA CORPORATION. + * Copyright (c) 2020-2023, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -26,6 +26,8 @@ #include #include +#include + template class SegmentedGatherTest : public cudf::test::BaseFixture { }; @@ -611,7 +613,7 @@ TEST_F(SegmentedGatherTestFloat, Fails) // Nulls are not supported in the gather map. EXPECT_THROW(cudf::lists::segmented_gather(cudf::lists_column_view{list}, cudf::lists_column_view{nulls_map}), - cudf::logic_error); + std::invalid_argument); // Gather map and list column sizes must be the same. EXPECT_THROW(cudf::lists::segmented_gather(cudf::lists_column_view{list}, diff --git a/cpp/tests/lists/combine/concatenate_list_elements_tests.cpp b/cpp/tests/lists/combine/concatenate_list_elements_tests.cpp index ca25560141c..77e8f904d01 100644 --- a/cpp/tests/lists/combine/concatenate_list_elements_tests.cpp +++ b/cpp/tests/lists/combine/concatenate_list_elements_tests.cpp @@ -1,5 +1,5 @@ /* - * Copyright (c) 2021, NVIDIA CORPORATION. + * Copyright (c) 2021-2023, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -21,6 +21,8 @@ #include +#include + using namespace cudf::test::iterators; namespace { @@ -47,13 +49,13 @@ TEST_F(ConcatenateListElementsTest, InvalidInput) // Input lists is not a 2-level depth lists column. { auto const col = IntCol{}; - EXPECT_THROW(cudf::lists::concatenate_list_elements(col), cudf::logic_error); + EXPECT_THROW(cudf::lists::concatenate_list_elements(col), std::invalid_argument); } // Input lists is not at least 2-level depth lists column. { auto const col = IntListsCol{1, 2, 3}; - EXPECT_THROW(cudf::lists::concatenate_list_elements(col), cudf::logic_error); + EXPECT_THROW(cudf::lists::concatenate_list_elements(col), std::invalid_argument); } } diff --git a/cpp/tests/lists/contains_tests.cpp b/cpp/tests/lists/contains_tests.cpp index 2139103500a..f592819dacb 100644 --- a/cpp/tests/lists/contains_tests.cpp +++ b/cpp/tests/lists/contains_tests.cpp @@ -1,5 +1,5 @@ /* - * Copyright (c) 2021-2022, NVIDIA CORPORATION. + * Copyright (c) 2021-2023, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -503,10 +503,11 @@ TEST_F(ContainsTest, ScalarTypeRelatedExceptions) {{1, 2, 3}, {4, 5, 6}}}.release(); auto skey = create_scalar_search_key(10); - EXPECT_THROW(cudf::lists::contains(list_of_lists->view(), *skey), cudf::logic_error); + EXPECT_THROW(cudf::lists::contains(list_of_lists->view(), *skey), cudf::data_type_error); EXPECT_THROW(cudf::lists::index_of(list_of_lists->view(), *skey, FIND_FIRST), - cudf::logic_error); - EXPECT_THROW(cudf::lists::index_of(list_of_lists->view(), *skey, FIND_LAST), cudf::logic_error); + cudf::data_type_error); + EXPECT_THROW(cudf::lists::index_of(list_of_lists->view(), *skey, FIND_LAST), + cudf::data_type_error); } { // Search key must match list elements in type. @@ -517,9 +518,11 @@ TEST_F(ContainsTest, ScalarTypeRelatedExceptions) } .release(); auto skey = create_scalar_search_key("Hello, World!"); - EXPECT_THROW(cudf::lists::contains(list_of_ints->view(), *skey), cudf::logic_error); - EXPECT_THROW(cudf::lists::index_of(list_of_ints->view(), *skey, FIND_FIRST), cudf::logic_error); - EXPECT_THROW(cudf::lists::index_of(list_of_ints->view(), *skey, FIND_LAST), cudf::logic_error); + EXPECT_THROW(cudf::lists::contains(list_of_ints->view(), *skey), cudf::data_type_error); + EXPECT_THROW(cudf::lists::index_of(list_of_ints->view(), *skey, FIND_FIRST), + cudf::data_type_error); + EXPECT_THROW(cudf::lists::index_of(list_of_ints->view(), *skey, FIND_LAST), + cudf::data_type_error); } } @@ -813,9 +816,11 @@ TEST_F(ContainsTest, VectorTypeRelatedExceptions) {{1, 2, 3}, {4, 5, 6}}}.release(); auto skey = cudf::test::fixed_width_column_wrapper{0, 1, 2}; - EXPECT_THROW(cudf::lists::contains(list_of_lists->view(), skey), cudf::logic_error); - EXPECT_THROW(cudf::lists::index_of(list_of_lists->view(), skey, FIND_FIRST), cudf::logic_error); - EXPECT_THROW(cudf::lists::index_of(list_of_lists->view(), skey, FIND_LAST), cudf::logic_error); + EXPECT_THROW(cudf::lists::contains(list_of_lists->view(), skey), cudf::data_type_error); + EXPECT_THROW(cudf::lists::index_of(list_of_lists->view(), skey, FIND_FIRST), + cudf::data_type_error); + EXPECT_THROW(cudf::lists::index_of(list_of_lists->view(), skey, FIND_LAST), + cudf::data_type_error); } { // Search key must match list elements in type. @@ -826,9 +831,11 @@ TEST_F(ContainsTest, VectorTypeRelatedExceptions) } .release(); auto skey = cudf::test::strings_column_wrapper{"Hello", "World"}; - EXPECT_THROW(cudf::lists::contains(list_of_ints->view(), skey), cudf::logic_error); - EXPECT_THROW(cudf::lists::index_of(list_of_ints->view(), skey, FIND_FIRST), cudf::logic_error); - EXPECT_THROW(cudf::lists::index_of(list_of_ints->view(), skey, FIND_LAST), cudf::logic_error); + EXPECT_THROW(cudf::lists::contains(list_of_ints->view(), skey), cudf::data_type_error); + EXPECT_THROW(cudf::lists::index_of(list_of_ints->view(), skey, FIND_FIRST), + cudf::data_type_error); + EXPECT_THROW(cudf::lists::index_of(list_of_ints->view(), skey, FIND_LAST), + cudf::data_type_error); } { // Search key column size must match lists column size. diff --git a/cpp/tests/strings/json_tests.cpp b/cpp/tests/strings/json_tests.cpp index 1924d809743..4a485de2f2a 100644 --- a/cpp/tests/strings/json_tests.cpp +++ b/cpp/tests/strings/json_tests.cpp @@ -1,5 +1,5 @@ /* - * Copyright (c) 2021-2022, NVIDIA CORPORATION. + * Copyright (c) 2021-2023, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -22,6 +22,8 @@ #include #include +#include + // reference: https://jsonpath.herokuapp.com/ // clang-format off @@ -566,7 +568,7 @@ TEST_F(JsonPathTests, GetJsonObjectIllegalQuery) auto query = [&]() { auto result = cudf::strings::get_json_object(cudf::strings_column_view(input), json_path); }; - EXPECT_THROW(query(), cudf::logic_error); + EXPECT_THROW(query(), std::invalid_argument); } { @@ -575,7 +577,7 @@ TEST_F(JsonPathTests, GetJsonObjectIllegalQuery) auto query = [&]() { auto result = cudf::strings::get_json_object(cudf::strings_column_view(input), json_path); }; - EXPECT_THROW(query(), cudf::logic_error); + EXPECT_THROW(query(), std::invalid_argument); } { @@ -584,7 +586,7 @@ TEST_F(JsonPathTests, GetJsonObjectIllegalQuery) auto query = [&]() { auto result = cudf::strings::get_json_object(cudf::strings_column_view(input), json_path); }; - EXPECT_THROW(query(), cudf::logic_error); + EXPECT_THROW(query(), std::invalid_argument); } } diff --git a/docs/cudf/source/developer_guide/contributing_guide.md b/docs/cudf/source/developer_guide/contributing_guide.md index bb3479cf4c1..b5ea9519842 100644 --- a/docs/cudf/source/developer_guide/contributing_guide.md +++ b/docs/cudf/source/developer_guide/contributing_guide.md @@ -123,19 +123,13 @@ There is no need to mention when the argument will be supported in the future. ### Handling libcudf Exceptions -Currently libcudf raises `cudf::logic_error` and `cudf::cuda_error`. -These error types are mapped to `RuntimeError` in python. -Several APIs use the exception payload `what()` message to determine the exception type raised by libcudf. - -Determining error type based on exception payload is brittle since libcudf does not maintain API stability on exception messages. -This is a compromise due to libcudf only raising a limited number of error types. -Only adopt this strategy when necessary. - -The projected roadmap is to diversify the exception types raised by libcudf. Standard C++ natively supports various [exception types](https://en.cppreference.com/w/cpp/error/exception), which Cython maps to [these Python exception types](https://docs.cython.org/en/latest/src/userguide/wrapping_CPlusPlus.html#exceptions). -In the future, libcudf may employ custom C++ exception types. -If that occurs, this section will be updated to reflect how these may be mapped to desired Python exception types. +In addition to built-in exceptions, libcudf also raises a few additional types of exceptions. +cuDF extends Cython's default mapping to account for these exception types. +When a new libcudf exception type is added, a suitable except clause should be added to cuDF's +[exception handler](https://github.com/rapidsai/cudf/blob/main/python/cudf/cudf/_lib/cpp/exception_handler.hpp). +If no built-in Python exception seems like a good match, a new Python exception should be created. ### Raising warnings diff --git a/java/src/main/native/include/jni_utils.hpp b/java/src/main/native/include/jni_utils.hpp index 78239b86ae2..ee2325cc76f 100644 --- a/java/src/main/native/include/jni_utils.hpp +++ b/java/src/main/native/include/jni_utils.hpp @@ -1,5 +1,5 @@ /* - * Copyright (c) 2019-2022, NVIDIA CORPORATION. + * Copyright (c) 2019-2023, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -32,6 +32,7 @@ constexpr jint MINIMUM_JNI_VERSION = JNI_VERSION_1_6; constexpr char const *CUDA_ERROR_CLASS = "ai/rapids/cudf/CudaException"; constexpr char const *CUDA_FATAL_ERROR_CLASS = "ai/rapids/cudf/CudaFatalException"; constexpr char const *CUDF_ERROR_CLASS = "ai/rapids/cudf/CudfException"; +constexpr char const *CUDF_DTYPE_ERROR_CLASS = "ai/rapids/cudf/CudfException"; constexpr char const *INDEX_OOB_CLASS = "java/lang/ArrayIndexOutOfBoundsException"; constexpr char const *ILLEGAL_ARG_CLASS = "java/lang/IllegalArgumentException"; constexpr char const *NPE_CLASS = "java/lang/NullPointerException"; @@ -861,6 +862,9 @@ inline void jni_cuda_check(JNIEnv *const env, cudaError_t cuda_status) { catch (const cudf::cuda_error &e) { \ JNI_CHECK_CUDA_ERROR(env, cudf::jni::CUDA_ERROR_CLASS, e, ret_val); \ } \ + catch (const cudf::data_type_error &e) { \ + JNI_CHECK_THROW_NEW(env, cudf::jni::CUDF_DTYPE_ERROR_CLASS, e.what(), ret_val); \ + } \ catch (const std::exception &e) { \ /* Double check whether the thrown exception is unrecoverable CUDA error or not. */ \ /* Like cudf::detail::throw_cuda_error, it is nearly certain that a fatal error */ \ diff --git a/python/cudf/cudf/_lib/CMakeLists.txt b/python/cudf/cudf/_lib/CMakeLists.txt index 4b785563484..f7d4f12ad81 100644 --- a/python/cudf/cudf/_lib/CMakeLists.txt +++ b/python/cudf/cudf/_lib/CMakeLists.txt @@ -62,6 +62,11 @@ rapids_cython_create_modules( LINKED_LIBRARIES "${linked_libraries}" ASSOCIATED_TARGETS cudf ) +# All modules need to include the header containing the exception handler. +foreach(target IN LISTS RAPIDS_CYTHON_CREATED_TARGETS) + target_include_directories(${target} PRIVATE ${CMAKE_CURRENT_LIST_DIR}) +endforeach() + target_link_libraries(strings_udf cudf_strings_udf) # TODO: Finding NumPy currently requires finding Development due to a bug in CMake. This bug was diff --git a/python/cudf/cudf/_lib/cpp/copying.pxd b/python/cudf/cudf/_lib/cpp/copying.pxd index bc89d364004..09e8538ebb7 100644 --- a/python/cudf/cudf/_lib/cpp/copying.pxd +++ b/python/cudf/cudf/_lib/cpp/copying.pxd @@ -1,4 +1,4 @@ -# Copyright (c) 2020-2022, NVIDIA CORPORATION. +# Copyright (c) 2020-2023, NVIDIA CORPORATION. from libc.stdint cimport int32_t, int64_t, uint8_t from libcpp cimport bool @@ -14,6 +14,7 @@ from cudf._lib.cpp.scalar.scalar cimport scalar from cudf._lib.cpp.table.table cimport table from cudf._lib.cpp.table.table_view cimport table_view from cudf._lib.cpp.types cimport size_type +from cudf._lib.exception_handler cimport cudf_exception_handler ctypedef const scalar constscalar @@ -32,7 +33,7 @@ cdef extern from "cudf/copying.hpp" namespace "cudf" nogil: const table_view& source_table, const column_view& gather_map, out_of_bounds_policy policy - ) except + + ) except +cudf_exception_handler cdef unique_ptr[column] shift( const column_view& input, diff --git a/python/cudf/cudf/_lib/cpp/lists/contains.pxd b/python/cudf/cudf/_lib/cpp/lists/contains.pxd index e3cb01721a0..e86c73deed2 100644 --- a/python/cudf/cudf/_lib/cpp/lists/contains.pxd +++ b/python/cudf/cudf/_lib/cpp/lists/contains.pxd @@ -1,4 +1,4 @@ -# Copyright (c) 2021-2022, NVIDIA CORPORATION. +# Copyright (c) 2021-2023, NVIDIA CORPORATION. from libcpp.memory cimport unique_ptr @@ -6,20 +6,21 @@ from cudf._lib.cpp.column.column cimport column from cudf._lib.cpp.column.column_view cimport column_view from cudf._lib.cpp.lists.lists_column_view cimport lists_column_view from cudf._lib.cpp.scalar.scalar cimport scalar +from cudf._lib.exception_handler cimport cudf_exception_handler cdef extern from "cudf/lists/contains.hpp" namespace "cudf::lists" nogil: cdef unique_ptr[column] contains( lists_column_view lists, scalar search_key, - ) except + + ) except +cudf_exception_handler cdef unique_ptr[column] index_of( lists_column_view lists, scalar search_key, - ) except + + ) except +cudf_exception_handler cdef unique_ptr[column] index_of( lists_column_view lists, column_view search_keys, - ) except + + ) except +cudf_exception_handler diff --git a/python/cudf/cudf/_lib/exception_handler.hpp b/python/cudf/cudf/_lib/exception_handler.hpp new file mode 100644 index 00000000000..8daffddd7bd --- /dev/null +++ b/python/cudf/cudf/_lib/exception_handler.hpp @@ -0,0 +1,80 @@ +/* + * Copyright (c) 2023, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#pragma once + +#include +#include +#include +#include + +namespace cudf_python { +namespace exceptions { + +/** + * @brief Exception handler to map C++ exceptions to Python ones in Cython + * + * This exception handler extends the base exception handler provided by + * Cython (https://github.com/cython/cython/blob/master/Cython/Utility/CppSupport.cpp#L9). + * In addition to the exceptions that Cython itself supports, this file adds support + * for additional exceptions thrown by libcudf that need to be mapped to specific Python + * exceptions. + * + * Since this function interoperates with Python's exception state, it does not throw + * any C++ exceptions. + */ +void cudf_exception_handler() +{ + // Catch a handful of different errors here and turn them into the + // equivalent Python errors. + try { + if (PyErr_Occurred()) + ; // let the latest Python exn pass through and ignore the current one else + throw; + } catch (const std::bad_alloc& exn) { + PyErr_SetString(PyExc_MemoryError, exn.what()); + } catch (const std::bad_cast& exn) { + PyErr_SetString(PyExc_TypeError, exn.what()); + } catch (const std::domain_error& exn) { + PyErr_SetString(PyExc_ValueError, exn.what()); + } catch (const cudf::data_type_error& exn) { + // Have to catch data_type_error before invalid_argument because it is a subclass + PyErr_SetString(PyExc_TypeError, exn.what()); + } catch (const std::invalid_argument& exn) { + PyErr_SetString(PyExc_ValueError, exn.what()); + } catch (const std::ios_base::failure& exn) { + // Unfortunately, in standard C++ we have no way of distinguishing EOF + // from other errors here; be careful with the exception mask + PyErr_SetString(PyExc_IOError, exn.what()); + } catch (const std::out_of_range& exn) { + // Change out_of_range to IndexError + PyErr_SetString(PyExc_IndexError, exn.what()); + } catch (const std::overflow_error& exn) { + PyErr_SetString(PyExc_OverflowError, exn.what()); + } catch (const std::range_error& exn) { + PyErr_SetString(PyExc_ArithmeticError, exn.what()); + } catch (const std::underflow_error& exn) { + PyErr_SetString(PyExc_ArithmeticError, exn.what()); + // The below is the default catch-all case. + } catch (const std::exception& exn) { + PyErr_SetString(PyExc_RuntimeError, exn.what()); + } catch (...) { + PyErr_SetString(PyExc_RuntimeError, "Unknown exception"); + } +} + +} // namespace exceptions +} // namespace cudf_python diff --git a/python/cudf/cudf/_lib/exception_handler.pxd b/python/cudf/cudf/_lib/exception_handler.pxd new file mode 100644 index 00000000000..14ac3bb1d40 --- /dev/null +++ b/python/cudf/cudf/_lib/exception_handler.pxd @@ -0,0 +1,5 @@ +# Copyright (c) 2023, NVIDIA CORPORATION. + + +cdef extern from "exception_handler.hpp" namespace "cudf_python::exceptions": + cdef void cudf_exception_handler() diff --git a/python/cudf/cudf/core/column/column.py b/python/cudf/cudf/core/column/column.py index fb1bcf6d673..b5f36aa3594 100644 --- a/python/cudf/cudf/core/column/column.py +++ b/python/cudf/cudf/core/column/column.py @@ -2587,19 +2587,10 @@ def concat_columns(objs: "MutableSequence[ColumnBase]") -> ColumnBase: f"size > {libcudf.MAX_COLUMN_SIZE_STR}" ) elif newsize == 0: - col = column_empty(0, head.dtype, masked=True) - else: - # Filter out inputs that have 0 length, then concatenate. - objs = [o for o in objs if len(o)] - try: - col = libcudf.concat.concat_columns(objs) - except RuntimeError as e: - if "exceeds size_type range" in str(e): - raise OverflowError( - "total size of output is too large for a cudf column" - ) from e - raise - return col + return column_empty(0, head.dtype, masked=True) + + # Filter out inputs that have 0 length, then concatenate. + return libcudf.concat.concat_columns([o for o in objs if len(o)]) def _proxy_cai_obj(cai, owner): diff --git a/python/cudf/cudf/core/column/decimal.py b/python/cudf/cudf/core/column/decimal.py index 157bc1f4291..96b8002e2a1 100644 --- a/python/cudf/cudf/core/column/decimal.py +++ b/python/cudf/cudf/core/column/decimal.py @@ -1,4 +1,4 @@ -# Copyright (c) 2021-2022, NVIDIA CORPORATION. +# Copyright (c) 2021-2023, NVIDIA CORPORATION. import warnings from decimal import Decimal @@ -79,32 +79,26 @@ def _binaryop(self, other: ColumnBinaryOperand, op: str): # Binary Arithmetics between decimal columns. `Scale` and `precision` # are computed outside of libcudf - unsupported_msg = ( - f"{op} not supported for the following dtypes: " - f"{self.dtype}, {other.dtype}" - ) - try: - if op in {"__add__", "__sub__", "__mul__", "__div__"}: - output_type = _get_decimal_type(lhs.dtype, rhs.dtype, op) - result = libcudf.binaryop.binaryop(lhs, rhs, op, output_type) - # TODO: Why is this necessary? Why isn't the result's - # precision already set correctly based on output_type? - result.dtype.precision = output_type.precision - elif op in { - "__eq__", - "__ne__", - "__lt__", - "__gt__", - "__le__", - "__ge__", - }: - result = libcudf.binaryop.binaryop(lhs, rhs, op, bool) - else: - raise TypeError(unsupported_msg) - except RuntimeError as e: - if "Unsupported operator for these types" in str(e): - raise TypeError(unsupported_msg) from e - raise + if op in {"__add__", "__sub__", "__mul__", "__div__"}: + output_type = _get_decimal_type(lhs.dtype, rhs.dtype, op) + result = libcudf.binaryop.binaryop(lhs, rhs, op, output_type) + # TODO: Why is this necessary? Why isn't the result's + # precision already set correctly based on output_type? + result.dtype.precision = output_type.precision + elif op in { + "__eq__", + "__ne__", + "__lt__", + "__gt__", + "__le__", + "__ge__", + }: + result = libcudf.binaryop.binaryop(lhs, rhs, op, bool) + else: + raise TypeError( + f"{op} not supported for the following dtypes: " + f"{self.dtype}, {other.dtype}" + ) return result diff --git a/python/cudf/cudf/core/column/lists.py b/python/cudf/cudf/core/column/lists.py index 9b64f26f0fb..4eea64c00d3 100644 --- a/python/cudf/cudf/core/column/lists.py +++ b/python/cudf/cudf/core/column/lists.py @@ -406,19 +406,9 @@ def contains(self, search_key: ScalarLike) -> ParentType: Series([False, True, True]) dtype: bool """ - search_key = cudf.Scalar(search_key) - try: - res = self._return_or_inplace( - contains_scalar(self._column, search_key) - ) - except RuntimeError as e: - if ( - "Type/Scale of search key does not " - "match list column element type." in str(e) - ): - raise TypeError(str(e)) from e - raise - return res + return self._return_or_inplace( + contains_scalar(self._column, cudf.Scalar(search_key)) + ) def index(self, search_key: Union[ScalarLike, ColumnLike]) -> ParentType: """ @@ -465,23 +455,14 @@ def index(self, search_key: Union[ScalarLike, ColumnLike]) -> ParentType: dtype: int32 """ - try: - if is_scalar(search_key): - return self._return_or_inplace( - index_of_scalar(self._column, cudf.Scalar(search_key)) - ) - else: - return self._return_or_inplace( - index_of_column(self._column, as_column(search_key)) - ) - - except RuntimeError as e: - if ( - "Type/Scale of search key does not " - "match list column element type." in str(e) - ): - raise TypeError(str(e)) from e - raise + if is_scalar(search_key): + return self._return_or_inplace( + index_of_scalar(self._column, cudf.Scalar(search_key)) + ) + else: + return self._return_or_inplace( + index_of_column(self._column, as_column(search_key)) + ) @property def leaves(self) -> ParentType: @@ -577,16 +558,9 @@ def take(self, lists_indices: ColumnLike) -> ParentType: "lists_indices should be column of values of index types." ) - try: - res = self._return_or_inplace( - segmented_gather(self._column, lists_indices_col) - ) - except RuntimeError as e: - if "contains nulls" in str(e): - raise ValueError("lists_indices contains null.") from e - raise - else: - return res + return self._return_or_inplace( + segmented_gather(self._column, lists_indices_col) + ) def unique(self) -> ParentType: """ @@ -720,16 +694,9 @@ def concat(self, dropna=True) -> ParentType: 1 [6.0, nan, 7.0, 8.0, 9.0] dtype: list """ - try: - result = concatenate_list_elements(self._column, dropna=dropna) - except RuntimeError as e: - if "Rows of the input column must be lists." in str(e): - raise ValueError( - "list.concat() can only be called on " - "list columns with at least one level " - "of nesting" - ) - return self._return_or_inplace(result) + return self._return_or_inplace( + concatenate_list_elements(self._column, dropna=dropna) + ) def astype(self, dtype): """ diff --git a/python/cudf/cudf/core/column/string.py b/python/cudf/cudf/core/column/string.py index 8d6ffe48957..d5ef5fb5d11 100644 --- a/python/cudf/cudf/core/column/string.py +++ b/python/cudf/cudf/core/column/string.py @@ -2379,29 +2379,18 @@ def get_json_object( dtype: object """ - try: - options = libstrings.GetJsonObjectOptions( - allow_single_quotes=allow_single_quotes, - strip_quotes_from_single_strings=( - strip_quotes_from_single_strings - ), - missing_fields_as_nulls=missing_fields_as_nulls, - ) - res = self._return_or_inplace( - libstrings.get_json_object( - self._column, cudf.Scalar(json_path, "str"), options - ) - ) - except RuntimeError as e: - matches = ( - "Unrecognized JSONPath operator", - "Invalid empty name in JSONPath query string", + options = libstrings.GetJsonObjectOptions( + allow_single_quotes=allow_single_quotes, + strip_quotes_from_single_strings=( + strip_quotes_from_single_strings + ), + missing_fields_as_nulls=missing_fields_as_nulls, + ) + return self._return_or_inplace( + libstrings.get_json_object( + self._column, cudf.Scalar(json_path, "str"), options ) - if any(match in str(e) for match in matches): - raise ValueError("JSONPath value not found") from e - raise - else: - return res + ) def split( self, diff --git a/python/cudf/cudf/tests/test_column.py b/python/cudf/cudf/tests/test_column.py index 7d113bbb9e2..a15afa727c0 100644 --- a/python/cudf/cudf/tests/test_column.py +++ b/python/cudf/cudf/tests/test_column.py @@ -520,10 +520,7 @@ def test_concatenate_large_column_strings(): s_1 = cudf.Series(["very long string " * string_scale_f] * num_strings) s_2 = cudf.Series(["very long string " * string_scale_f] * num_strings) - with pytest.raises( - OverflowError, - match="total size of output is too large for a cudf column", - ): + with pytest.raises(OverflowError): cudf.concat([s_1, s_2]) From ac1cac67839ceb2272e1eba151782cfaa744270a Mon Sep 17 00:00:00 2001 From: Raza Jafri Date: Sun, 26 Feb 2023 20:29:11 -0800 Subject: [PATCH 19/60] Add JNI methods for detecting and purging non-empty nulls from LIST and STRUCT (#12742) This PR adds methods for detecting and purging non-empty nulls. Authors: - Raza Jafri (https://github.com/razajafri) - Nghia Truong (https://github.com/ttnghia) - AJ Schmidt (https://github.com/ajschmidt8) Approvers: - Nghia Truong (https://github.com/ttnghia) URL: https://github.com/rapidsai/cudf/pull/12742 --- .../main/java/ai/rapids/cudf/ColumnView.java | 37 +++++++++++ java/src/main/native/src/ColumnViewJni.cpp | 22 +++++++ .../java/ai/rapids/cudf/ColumnVectorTest.java | 62 +++++++++++++++++++ 3 files changed, 121 insertions(+) diff --git a/java/src/main/java/ai/rapids/cudf/ColumnView.java b/java/src/main/java/ai/rapids/cudf/ColumnView.java index 0cb9ed37d9f..84183819854 100644 --- a/java/src/main/java/ai/rapids/cudf/ColumnView.java +++ b/java/src/main/java/ai/rapids/cudf/ColumnView.java @@ -4639,6 +4639,10 @@ static native long makeCudfColumnView(int type, int scale, long data, long dataS static native long applyBooleanMask(long arrayColumnView, long booleanMaskHandle) throws CudfException; + static native boolean hasNonEmptyNulls(long handle) throws CudfException; + + static native long purgeNonEmptyNulls(long handle) throws CudfException; + /** * A utility class to create column vector like objects without refcounts and other APIs when * creating the device side vector from host side nested vectors. Eventually this can go away or @@ -4997,4 +5001,37 @@ public HostColumnVector copyToHost() { } } } + + /** + * Exact check if a column or its descendants have non-empty null rows + * + * @return Whether the column or its descendants have non-empty null rows + */ + public boolean hasNonEmptyNulls() { + return hasNonEmptyNulls(viewHandle); + } + + /** + * Copies this column into output while purging any non-empty null rows in the column or its + * descendants. + * + * If this column is not of compound type (LIST/STRING/STRUCT/DICTIONARY), the output will be + * the same as input. + * + * The purge operation only applies directly to LIST and STRING columns, but it applies indirectly + * to STRUCT/DICTIONARY columns as well, since these columns may have child columns that + * are LIST or STRING. + * + * Examples: + * lists = data: [{{0,1}, {2,3}, {4,5}} validity: {true, false, true}] + * lists[1] is null, but the list's child column still stores `{2,3}`. + * + * After purging the contents of the list's null rows, the column's contents will be: + * lists = [data: {{0,1}, {4,5}} validity: {true, false, true}] + * + * @return A new column with equivalent contents to `input`, but with null rows purged + */ + public ColumnVector purgeNonEmptyNulls() { + return new ColumnVector(purgeNonEmptyNulls(viewHandle)); + } } diff --git a/java/src/main/native/src/ColumnViewJni.cpp b/java/src/main/native/src/ColumnViewJni.cpp index c42cc430560..f2c361c5e8c 100644 --- a/java/src/main/native/src/ColumnViewJni.cpp +++ b/java/src/main/native/src/ColumnViewJni.cpp @@ -2457,4 +2457,26 @@ JNIEXPORT jlong JNICALL Java_ai_rapids_cudf_ColumnView_applyBooleanMask( CATCH_STD(env, 0); } +JNIEXPORT jboolean JNICALL +Java_ai_rapids_cudf_ColumnView_hasNonEmptyNulls(JNIEnv *env, jclass, jlong column_view_handle) { + JNI_NULL_CHECK(env, column_view_handle, "column_view handle is null", 0); + try { + cudf::jni::auto_set_device(env); + auto const *cv = reinterpret_cast(column_view_handle); + return cudf::has_nonempty_nulls(*cv); + } + CATCH_STD(env, 0); +} + +JNIEXPORT jlong JNICALL +Java_ai_rapids_cudf_ColumnView_purgeNonEmptyNulls(JNIEnv *env, jclass, jlong column_view_handle) { + JNI_NULL_CHECK(env, column_view_handle, "column_view handle is null", 0); + try { + cudf::jni::auto_set_device(env); + auto const *cv = reinterpret_cast(column_view_handle); + return release_as_jlong(cudf::purge_nonempty_nulls(*cv)); + } + CATCH_STD(env, 0); +} + } // extern "C" diff --git a/java/src/test/java/ai/rapids/cudf/ColumnVectorTest.java b/java/src/test/java/ai/rapids/cudf/ColumnVectorTest.java index 937077c89c9..7848807dab8 100644 --- a/java/src/test/java/ai/rapids/cudf/ColumnVectorTest.java +++ b/java/src/test/java/ai/rapids/cudf/ColumnVectorTest.java @@ -32,6 +32,7 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; +import java.util.Optional; import java.util.function.Supplier; import java.util.stream.Collectors; import java.util.stream.IntStream; @@ -6691,4 +6692,65 @@ void testApplyBooleanMaskFromListOfStructure() { assertColumnsAreEqual(expectedCv, actualCv); } } + + /** + * The caller needs to make sure to close the returned ColumnView + */ + private ColumnView[] getColumnViewWithNonEmptyNulls() { + List list0 = Arrays.asList(1, 2, 3); + List list1 = Arrays.asList(4, 5, null); + List list2 = Arrays.asList(7, 8, 9); + List list3 = null; + ColumnVector input = makeListsColumn(DType.INT32, list0, list1, list2, list3); + // Modify the validity buffer + BaseDeviceMemoryBuffer dmb = input.getDeviceBufferFor(BufferType.VALIDITY); + try (HostMemoryBuffer newValidity = HostMemoryBuffer.allocate(64)) { + newValidity.copyFromDeviceBuffer(dmb); + BitVectorHelper.setNullAt(newValidity, 1); + dmb.copyFromHostBuffer(newValidity); + } + try (HostColumnVector hostColumnVector = input.copyToHost()) { + assert (hostColumnVector.isNull(1)); + assert (hostColumnVector.isNull(3)); + } + try (ColumnVector expectedOffsetsBeforePurge = ColumnVector.fromInts(0, 3, 6, 9, 9)) { + ColumnView offsetsCvBeforePurge = input.getListOffsetsView(); + assertColumnsAreEqual(expectedOffsetsBeforePurge, offsetsCvBeforePurge); + } + ColumnView colWithNonEmptyNulls = new ColumnView(input.type, input.rows, Optional.of(2L), dmb, + input.getDeviceBufferFor(BufferType.OFFSET), input.getChildColumnViews()); + assertEquals(2, colWithNonEmptyNulls.nullCount); + return new ColumnView[]{input, colWithNonEmptyNulls}; + } + + @Test + void testPurgeNonEmptyNullsList() { + ColumnView[] values = getColumnViewWithNonEmptyNulls(); + try (ColumnView colWithNonEmptyNulls = values[1]; + ColumnView input = values[0]; + // purge non-empty nulls + ColumnView colWithEmptyNulls = colWithNonEmptyNulls.purgeNonEmptyNulls(); + ColumnVector expectedOffsetsAfterPurge = ColumnVector.fromInts(0, 3, 3, 6, 6); + ColumnView offsetsCvAfterPurge = colWithEmptyNulls.getListOffsetsView()) { + assertTrue(colWithNonEmptyNulls.hasNonEmptyNulls()); + assertColumnsAreEqual(expectedOffsetsAfterPurge, offsetsCvAfterPurge); + assertFalse(colWithEmptyNulls.hasNonEmptyNulls()); + } + } + + @Test + void testPurgeNonEmptyNullsStruct() { + ColumnView[] values = getColumnViewWithNonEmptyNulls(); + try (ColumnView listCol = values[1]; + ColumnView input = values[0]; + ColumnView stringsCol = ColumnVector.fromStrings("A", "col", "of", "Strings"); + ColumnView structView = ColumnView.makeStructView(stringsCol, listCol); + ColumnView structWithEmptyNulls = structView.purgeNonEmptyNulls(); + ColumnView newListChild = structWithEmptyNulls.getChildColumnView(1); + ColumnVector expectedOffsetsAfterPurge = ColumnVector.fromInts(0, 3, 3, 6, 6); + ColumnView offsetsCvAfterPurge = newListChild.getListOffsetsView()) { + assertColumnsAreEqual(expectedOffsetsAfterPurge, offsetsCvAfterPurge); + assertFalse(newListChild.hasNonEmptyNulls()); + } + } } From 202578307c81b1f7aad8fbe6f71ca9c7f3fa5c4a Mon Sep 17 00:00:00 2001 From: David Wendt <45795991+davidwendt@users.noreply.github.com> Date: Mon, 27 Feb 2023 13:46:45 -0500 Subject: [PATCH 20/60] Improve performance for cudf::strings::count_characters for long strings (#12779) Adds more efficient counting algorithm specifically for columns with long strings--greater than 64 bytes on average. The internal detail method will be used to help improve performance in other strings functions. Authors: - David Wendt (https://github.com/davidwendt) - Bradley Dice (https://github.com/bdice) Approvers: - Nghia Truong (https://github.com/ttnghia) - Bradley Dice (https://github.com/bdice) URL: https://github.com/rapidsai/cudf/pull/12779 --- cpp/benchmarks/CMakeLists.txt | 2 +- cpp/benchmarks/string/lengths.cpp | 56 +++++++++++++++++++ cpp/src/strings/attributes.cu | 91 +++++++++++++++++++++++++++++-- cpp/tests/strings/attrs_tests.cpp | 22 ++++++-- 4 files changed, 160 insertions(+), 11 deletions(-) create mode 100644 cpp/benchmarks/string/lengths.cpp diff --git a/cpp/benchmarks/CMakeLists.txt b/cpp/benchmarks/CMakeLists.txt index c5ae3345da5..11da30f108a 100644 --- a/cpp/benchmarks/CMakeLists.txt +++ b/cpp/benchmarks/CMakeLists.txt @@ -295,7 +295,7 @@ ConfigureBench( string/url_decode.cu ) -ConfigureNVBench(STRINGS_NVBENCH string/like.cpp string/reverse.cpp) +ConfigureNVBench(STRINGS_NVBENCH string/like.cpp string/reverse.cpp string/lengths.cpp) # ################################################################################################## # * json benchmark ------------------------------------------------------------------- diff --git a/cpp/benchmarks/string/lengths.cpp b/cpp/benchmarks/string/lengths.cpp new file mode 100644 index 00000000000..4540e4a8f42 --- /dev/null +++ b/cpp/benchmarks/string/lengths.cpp @@ -0,0 +1,56 @@ +/* + * Copyright (c) 2023, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include +#include + +#include +#include +#include + +#include + +static void bench_lengths(nvbench::state& state) +{ + auto const num_rows = static_cast(state.get_int64("num_rows")); + auto const row_width = static_cast(state.get_int64("row_width")); + + if (static_cast(num_rows) * static_cast(row_width) >= + static_cast(std::numeric_limits::max())) { + state.skip("Skip benchmarks greater than size_type limit"); + } + + data_profile const table_profile = data_profile_builder().distribution( + cudf::type_id::STRING, distribution_id::NORMAL, 0, row_width); + auto const table = + create_random_table({cudf::type_id::STRING}, row_count{num_rows}, table_profile); + cudf::strings_column_view input(table->view().column(0)); + + state.set_cuda_stream(nvbench::make_cuda_stream_view(cudf::get_default_stream().value())); + // gather some throughput statistics as well + auto chars_size = input.chars_size(); + state.add_global_memory_reads(chars_size); // all bytes are read; + state.add_global_memory_writes(num_rows); // output is an integer per row + + state.exec(nvbench::exec_tag::sync, [&](nvbench::launch& launch) { + auto result = cudf::strings::count_characters(input); + }); +} + +NVBENCH_BENCH(bench_lengths) + .set_name("strings_lengths") + .add_int64_axis("num_rows", {4096, 32768, 262144, 2097152, 16777216}) + .add_int64_axis("row_width", {32, 64, 128, 256, 512, 1024, 2048, 4096}); diff --git a/cpp/src/strings/attributes.cu b/cpp/src/strings/attributes.cu index 127d3aa8fe7..66288c7d14d 100644 --- a/cpp/src/strings/attributes.cu +++ b/cpp/src/strings/attributes.cu @@ -1,5 +1,5 @@ /* - * Copyright (c) 2019-2022, NVIDIA CORPORATION. + * Copyright (c) 2019-2023, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -19,7 +19,9 @@ #include #include #include +#include #include +#include #include #include #include @@ -29,6 +31,7 @@ #include #include +#include #include #include #include @@ -37,10 +40,24 @@ #include #include +#include + namespace cudf { namespace strings { namespace detail { namespace { + +/** + * @brief Threshold to decide on using string or warp parallel functions. + * + * If the average byte length of a string in a column exceeds this value then + * the warp-parallel function is used. + * Otherwise, a regular string-parallel function is used. + * + * This value was found using the strings_lengths benchmark results. + */ +constexpr size_type AVG_CHAR_BYTES_THRESHOLD = 64; + /** * @brief Returns a numeric column containing lengths of each string in * based on the provided unary function. @@ -85,21 +102,85 @@ std::unique_ptr counts_fn(strings_column_view const& strings, return results; } +/** + * @brief Count characters using a warp per string + * + * @param d_strings Column with strings to count + * @param d_lengths Results of the counts per string + */ +__global__ void count_characters_parallel_fn(column_device_view const d_strings, + size_type* d_lengths) +{ + size_type const idx = static_cast(threadIdx.x + blockIdx.x * blockDim.x); + using warp_reduce = cub::WarpReduce; + __shared__ typename warp_reduce::TempStorage temp_storage; + + if (idx >= (d_strings.size() * cudf::detail::warp_size)) { return; } + + auto const str_idx = idx / cudf::detail::warp_size; + auto const lane_idx = idx % cudf::detail::warp_size; + if (d_strings.is_null(str_idx)) { + d_lengths[str_idx] = 0; + return; + } + auto const d_str = d_strings.element(str_idx); + auto const str_ptr = d_str.data(); + + auto count = 0; + for (auto i = lane_idx; i < d_str.size_bytes(); i += cudf::detail::warp_size) { + count += static_cast(is_begin_utf8_char(str_ptr[i])); + } + auto const char_count = warp_reduce(temp_storage).Sum(count); + if (lane_idx == 0) { d_lengths[str_idx] = char_count; } +} + +std::unique_ptr count_characters_parallel(strings_column_view const& input, + rmm::cuda_stream_view stream, + rmm::mr::device_memory_resource* mr) +{ + // create output column + auto results = make_numeric_column(data_type{type_to_id()}, + input.size(), + cudf::detail::copy_bitmask(input.parent(), stream, mr), + input.null_count(), + stream, + mr); + + auto const d_lengths = results->mutable_view().data(); + auto const d_strings = cudf::column_device_view::create(input.parent(), stream); + + // fill in the lengths + constexpr int block_size = 256; + cudf::detail::grid_1d grid{input.size() * cudf::detail::warp_size, block_size}; + count_characters_parallel_fn<<>>( + *d_strings, d_lengths); + + // reset null count after call to mutable_view() + results->set_null_count(input.null_count()); + + return results; +} + } // namespace -std::unique_ptr count_characters(strings_column_view const& strings, +std::unique_ptr count_characters(strings_column_view const& input, rmm::cuda_stream_view stream, rmm::mr::device_memory_resource* mr) { - auto ufn = [] __device__(const string_view& d_str) { return d_str.length(); }; - return counts_fn(strings, ufn, stream, mr); + if ((input.size() == input.null_count()) || + ((input.chars_size() / (input.size() - input.null_count())) < AVG_CHAR_BYTES_THRESHOLD)) { + auto ufn = [] __device__(string_view const& d_str) { return d_str.length(); }; + return counts_fn(input, ufn, stream, mr); + } + + return count_characters_parallel(input, stream, mr); } std::unique_ptr count_bytes(strings_column_view const& strings, rmm::cuda_stream_view stream, rmm::mr::device_memory_resource* mr) { - auto ufn = [] __device__(const string_view& d_str) { return d_str.size_bytes(); }; + auto ufn = [] __device__(string_view const& d_str) { return d_str.size_bytes(); }; return counts_fn(strings, ufn, stream, mr); } diff --git a/cpp/tests/strings/attrs_tests.cpp b/cpp/tests/strings/attrs_tests.cpp index 9ff2c55ed81..eff992604a6 100644 --- a/cpp/tests/strings/attrs_tests.cpp +++ b/cpp/tests/strings/attrs_tests.cpp @@ -1,5 +1,5 @@ /* - * Copyright (c) 2019-2021, NVIDIA CORPORATION. + * Copyright (c) 2019-2023, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -65,7 +65,7 @@ TEST_F(StringsAttributesTest, ZeroSizeStringsColumn) TEST_F(StringsAttributesTest, StringsLengths) { std::vector h_strings{ - "eee", "bb", nullptr, "", "aa", "ééé", " something a bit longer "}; + "eee", "bb", nullptr, "", "aa", "ééé", "something a bit longer than 32 bytes"}; cudf::test::strings_column_wrapper strings( h_strings.begin(), h_strings.end(), @@ -74,17 +74,16 @@ TEST_F(StringsAttributesTest, StringsLengths) { auto results = cudf::strings::count_characters(strings_view); - std::vector h_expected{3, 2, 0, 0, 2, 3, 24}; + std::vector h_expected{3, 2, 0, 0, 2, 3, 36}; cudf::test::fixed_width_column_wrapper expected( h_expected.begin(), h_expected.end(), thrust::make_transform_iterator(h_strings.begin(), [](auto str) { return str != nullptr; })); - CUDF_TEST_EXPECT_COLUMNS_EQUAL(*results, expected); } { auto results = cudf::strings::count_bytes(strings_view); - std::vector h_expected{3, 2, 0, 0, 2, 6, 24}; + std::vector h_expected{3, 2, 0, 0, 2, 6, 36}; cudf::test::fixed_width_column_wrapper expected( h_expected.begin(), h_expected.end(), @@ -93,3 +92,16 @@ TEST_F(StringsAttributesTest, StringsLengths) CUDF_TEST_EXPECT_COLUMNS_EQUAL(*results, expected); } } + +TEST_F(StringsAttributesTest, StringsLengthsLong) +{ + std::vector h_strings( + 40000, "something a bit longer than 32 bytes ééé ééé ééé ééé ééé ééé ééé"); + cudf::test::strings_column_wrapper strings(h_strings.begin(), h_strings.end()); + auto strings_view = cudf::strings_column_view(strings); + + auto results = cudf::strings::count_characters(strings_view); + std::vector h_expected(h_strings.size(), 64); + cudf::test::fixed_width_column_wrapper expected(h_expected.begin(), h_expected.end()); + CUDF_TEST_EXPECT_COLUMNS_EQUAL(*results, expected); +} From 9a91270b59f846cd4b0d6a1577183c6b333cdf57 Mon Sep 17 00:00:00 2001 From: Vyas Ramasubramani Date: Mon, 27 Feb 2023 16:11:22 -0800 Subject: [PATCH 21/60] Remove tokenizers pre-install pinning. (#12854) We pinned tokenizers during early wheel development because at the time the latest version on PyPI was only a source distribution without wheels available for arm and we didn't want to have to compile the package. More recent versions appear to be providing aarch binaries. Conversely, the pinned version (0.10.2) predated the existence of Python 3.10 wheels, which is resulting in nightly CI failing because Python 3.10 runs no longer find a wheel and attempt to compile from source. Authors: - Vyas Ramasubramani (https://github.com/vyasr) Approvers: - Bradley Dice (https://github.com/bdice) - AJ Schmidt (https://github.com/ajschmidt8) URL: https://github.com/rapidsai/cudf/pull/12854 --- .github/workflows/pr.yaml | 3 +-- .github/workflows/test.yaml | 2 +- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/.github/workflows/pr.yaml b/.github/workflows/pr.yaml index 3a80139e333..c36c539a102 100644 --- a/.github/workflows/pr.yaml +++ b/.github/workflows/pr.yaml @@ -113,8 +113,7 @@ jobs: build_type: pull-request package-name: cudf # Install cupy-cuda11x for arm from a special index url - # Install tokenizers last binary wheel to avoid a Rust compile from the latest sdist - test-before-arm64: "pip install tokenizers==0.10.2 cupy-cuda11x -f https://pip.cupy.dev/aarch64" + test-before-arm64: "pip install cupy-cuda11x -f https://pip.cupy.dev/aarch64" test-unittest: "pytest -v -n 8 ./python/cudf/cudf/tests" test-smoketest: "python ./ci/wheel_smoke_test_cudf.py" wheel-build-dask-cudf: diff --git a/.github/workflows/test.yaml b/.github/workflows/test.yaml index 4d9e97a7b28..5dc04ece919 100644 --- a/.github/workflows/test.yaml +++ b/.github/workflows/test.yaml @@ -86,7 +86,7 @@ jobs: date: ${{ inputs.date }} sha: ${{ inputs.sha }} package-name: cudf - test-before-arm64: "pip install tokenizers==0.10.2 cupy-cuda11x -f https://pip.cupy.dev/aarch64" + test-before-arm64: "pip install cupy-cuda11x -f https://pip.cupy.dev/aarch64" test-unittest: "pytest -v -n 8 ./python/cudf/cudf/tests" wheel-tests-dask-cudf: secrets: inherit From 8a45ae266641d96f30c9336abcd656d5290beaa1 Mon Sep 17 00:00:00 2001 From: Jake Awe <50372925+AyodeAwe@users.noreply.github.com> Date: Tue, 28 Feb 2023 09:10:18 -0600 Subject: [PATCH 22/60] Make docs builds less verbose (#12836) This PR adds the `--no-progress` flag to reduce verbose output from the `s3 sync` commands. Authors: - Jake Awe (https://github.com/AyodeAwe) Approvers: - AJ Schmidt (https://github.com/ajschmidt8) URL: https://github.com/rapidsai/cudf/pull/12836 --- ci/build_docs.sh | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/ci/build_docs.sh b/ci/build_docs.sh index 9551d98e9fe..6daedb59733 100755 --- a/ci/build_docs.sh +++ b/ci/build_docs.sh @@ -41,7 +41,8 @@ popd if [[ ${RAPIDS_BUILD_TYPE} == "branch" ]]; then - aws s3 sync --delete cpp/doxygen/html "s3://rapidsai-docs/libcudf/${VERSION_NUMBER}/html" - aws s3 sync --delete docs/cudf/_html "s3://rapidsai-docs/cudf/${VERSION_NUMBER}/html" - aws s3 sync --delete docs/cudf/_text "s3://rapidsai-docs/cudf/${VERSION_NUMBER}/txt" + rapids-logger "Upload Docs to S3" + aws s3 sync --no-progress --delete cpp/doxygen/html "s3://rapidsai-docs/libcudf/${VERSION_NUMBER}/html" + aws s3 sync --no-progress --delete docs/cudf/_html "s3://rapidsai-docs/cudf/${VERSION_NUMBER}/html" + aws s3 sync --no-progress --delete docs/cudf/_text "s3://rapidsai-docs/cudf/${VERSION_NUMBER}/txt" fi From a7e50920f35c667d041678f6a82d975355cdfd8f Mon Sep 17 00:00:00 2001 From: Peter Andreas Entschev Date: Tue, 28 Feb 2023 19:35:20 +0100 Subject: [PATCH 23/60] Update RMM allocators (#12861) Authors: - Peter Andreas Entschev (https://github.com/pentschev) Approvers: - Lawrence Mitchell (https://github.com/wence-) URL: https://github.com/rapidsai/cudf/pull/12861 --- python/cudf/cudf/__init__.py | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/python/cudf/cudf/__init__.py b/python/cudf/cudf/__init__.py index 04b64e18594..7bab131a85a 100644 --- a/python/cudf/cudf/__init__.py +++ b/python/cudf/cudf/__init__.py @@ -8,6 +8,8 @@ from numba import config as numba_config, cuda import rmm +from rmm.allocators.cupy import rmm_cupy_allocator +from rmm.allocators.numba import RMMNumbaManager from cudf import api, core, datasets, testing from cudf.api.extensions import ( @@ -96,8 +98,8 @@ del patch_numba_linker_if_needed -cuda.set_memory_manager(rmm.RMMNumbaManager) -cupy.cuda.set_allocator(rmm.rmm_cupy_allocator) +cuda.set_memory_manager(RMMNumbaManager) +cupy.cuda.set_allocator(rmm_cupy_allocator) try: # Numba 0.54: Disable low occupancy warnings From afdb51bb90969e8f2b7ec75dd79d8fb6c437d461 Mon Sep 17 00:00:00 2001 From: Karthikeyan <6488848+karthikeyann@users.noreply.github.com> Date: Thu, 2 Mar 2023 02:05:35 +0530 Subject: [PATCH 24/60] Adds JSON reader, writer io benchmark (#12753) - Add JSON writer benchmark. This benchmark is modeled after CSV writer. - Add JSON reader benchmark with file data source ([NESTED_JSON](https://github.com/rapidsai/cudf/blob/branch-23.04/cpp/benchmarks/io/json/nested_json.cpp?rgh-link-date=2023-02-08T22%3A43%3A38Z) only does parsing and only on device buffers). This benchmark is modeled after BM_csv_read_io fixes part of https://github.com/rapidsai/cudf/issues/12739 Authors: - Karthikeyan (https://github.com/karthikeyann) Approvers: - Vukasin Milovanovic (https://github.com/vuule) - David Wendt (https://github.com/davidwendt) URL: https://github.com/rapidsai/cudf/pull/12753 --- cpp/benchmarks/CMakeLists.txt | 3 +- cpp/benchmarks/io/json/json_reader_input.cpp | 128 +++++++++++++++++++ cpp/benchmarks/io/json/json_writer.cpp | 125 ++++++++++++++++++ cpp/src/io/json/write_json.cu | 4 + 4 files changed, 259 insertions(+), 1 deletion(-) create mode 100644 cpp/benchmarks/io/json/json_reader_input.cpp create mode 100644 cpp/benchmarks/io/json/json_writer.cpp diff --git a/cpp/benchmarks/CMakeLists.txt b/cpp/benchmarks/CMakeLists.txt index 11da30f108a..e6b59c0b9f0 100644 --- a/cpp/benchmarks/CMakeLists.txt +++ b/cpp/benchmarks/CMakeLists.txt @@ -301,7 +301,8 @@ ConfigureNVBench(STRINGS_NVBENCH string/like.cpp string/reverse.cpp string/lengt # * json benchmark ------------------------------------------------------------------- ConfigureBench(JSON_BENCH string/json.cu) ConfigureNVBench(FST_NVBENCH io/fst.cu) -ConfigureNVBench(NESTED_JSON_NVBENCH io/json/nested_json.cpp) +ConfigureNVBench(JSON_READER_NVBENCH io/json/nested_json.cpp io/json/json_reader_input.cpp) +ConfigureNVBench(JSON_WRITER_NVBENCH io/json/json_writer.cpp) # ################################################################################################## # * io benchmark --------------------------------------------------------------------- diff --git a/cpp/benchmarks/io/json/json_reader_input.cpp b/cpp/benchmarks/io/json/json_reader_input.cpp new file mode 100644 index 00000000000..55614d040d5 --- /dev/null +++ b/cpp/benchmarks/io/json/json_reader_input.cpp @@ -0,0 +1,128 @@ +/* + * Copyright (c) 2023, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include +#include +#include +#include + +#include +#include + +#include + +// Size of the data in the the benchmark dataframe; chosen to be low enough to allow benchmarks to +// run on most GPUs, but large enough to allow highest throughput +constexpr size_t data_size = 512 << 20; +constexpr cudf::size_type num_cols = 64; + +void json_read_common(cudf::io::json_writer_options const& write_opts, + cuio_source_sink_pair& source_sink, + nvbench::state& state) +{ + cudf::io::write_json(write_opts); + + cudf::io::json_reader_options read_opts = + cudf::io::json_reader_options::builder(source_sink.make_source_info()); + + auto mem_stats_logger = cudf::memory_stats_logger(); + state.set_cuda_stream(nvbench::make_cuda_stream_view(cudf::get_default_stream().value())); + state.exec(nvbench::exec_tag::sync | nvbench::exec_tag::timer, + [&](nvbench::launch& launch, auto& timer) { + try_drop_l3_cache(); + + timer.start(); + cudf::io::read_json(read_opts); + timer.stop(); + }); + + auto const time = state.get_summary("nv/cold/time/gpu/mean").get_float64("value"); + state.add_element_count(static_cast(data_size) / time, "bytes_per_second"); + state.add_buffer_size( + mem_stats_logger.peak_memory_usage(), "peak_memory_usage", "peak_memory_usage"); + state.add_buffer_size(source_sink.size(), "encoded_file_size", "encoded_file_size"); +} + +template +void BM_json_read_io(nvbench::state& state, nvbench::type_list>) +{ + auto const d_type = get_type_or_group({static_cast(data_type::INTEGRAL), + static_cast(data_type::FLOAT), + static_cast(data_type::DECIMAL), + static_cast(data_type::TIMESTAMP), + static_cast(data_type::DURATION), + static_cast(data_type::STRING), + static_cast(data_type::LIST), + static_cast(data_type::STRUCT)}); + + auto const source_type = IO; + + auto const tbl = create_random_table( + cycle_dtypes(d_type, num_cols), table_size_bytes{data_size}, data_profile_builder()); + auto const view = tbl->view(); + + cuio_source_sink_pair source_sink(source_type); + cudf::io::json_writer_options const write_opts = + cudf::io::json_writer_options::builder(source_sink.make_sink_info(), view).na_rep("null"); + + json_read_common(write_opts, source_sink, state); +} + +template +void BM_json_read_data_type( + nvbench::state& state, nvbench::type_list, nvbench::enum_type>) +{ + auto const d_type = get_type_or_group(static_cast(DataType)); + auto const source_type = IO; + + auto const tbl = create_random_table( + cycle_dtypes(d_type, num_cols), table_size_bytes{data_size}, data_profile_builder()); + auto const view = tbl->view(); + + cuio_source_sink_pair source_sink(source_type); + cudf::io::json_writer_options const write_opts = + cudf::io::json_writer_options::builder(source_sink.make_sink_info(), view).na_rep("null"); + + json_read_common(write_opts, source_sink, state); +} + +using d_type_list = nvbench::enum_type_list; + +using io_list = nvbench::enum_type_list; + +using compression_list = + nvbench::enum_type_list; + +NVBENCH_BENCH_TYPES(BM_json_read_data_type, + NVBENCH_TYPE_AXES(d_type_list, + nvbench::enum_type_list)) + .set_name("json_read_data_type") + .set_type_axes_names({"data_type", "io"}) + .set_min_samples(4); + +NVBENCH_BENCH_TYPES(BM_json_read_io, NVBENCH_TYPE_AXES(io_list)) + .set_name("json_read_io") + .set_type_axes_names({"io"}) + .set_min_samples(4); diff --git a/cpp/benchmarks/io/json/json_writer.cpp b/cpp/benchmarks/io/json/json_writer.cpp new file mode 100644 index 00000000000..ee183b327fe --- /dev/null +++ b/cpp/benchmarks/io/json/json_writer.cpp @@ -0,0 +1,125 @@ +/* + * Copyright (c) 2023, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include +#include +#include +#include + +#include +#include + +#include + +// Size of the data in the the benchmark dataframe; chosen to be low enough to allow benchmarks to +// run on most GPUs, but large enough to allow highest throughput +constexpr size_t data_size = 512 << 20; +constexpr cudf::size_type num_cols = 64; + +void json_write_common(cudf::io::json_writer_options const& write_opts, + cuio_source_sink_pair& source_sink, + size_t const data_size, + nvbench::state& state) +{ + auto mem_stats_logger = cudf::memory_stats_logger(); + state.set_cuda_stream(nvbench::make_cuda_stream_view(cudf::get_default_stream().value())); + state.exec(nvbench::exec_tag::sync | nvbench::exec_tag::timer, + [&](nvbench::launch& launch, auto& timer) { + try_drop_l3_cache(); + + timer.start(); + cudf::io::write_json(write_opts); + timer.stop(); + }); + + auto const time = state.get_summary("nv/cold/time/gpu/mean").get_float64("value"); + state.add_element_count(static_cast(data_size) / time, "bytes_per_second"); + state.add_buffer_size( + mem_stats_logger.peak_memory_usage(), "peak_memory_usage", "peak_memory_usage"); + state.add_buffer_size(source_sink.size(), "encoded_file_size", "encoded_file_size"); +} + +template +void BM_json_write_io(nvbench::state& state, nvbench::type_list>) +{ + auto const d_type = get_type_or_group({static_cast(data_type::INTEGRAL), + static_cast(data_type::FLOAT), + static_cast(data_type::DECIMAL), + static_cast(data_type::TIMESTAMP), + static_cast(data_type::DURATION), + static_cast(data_type::STRING), + static_cast(data_type::LIST), + static_cast(data_type::STRUCT)}); + + auto const source_type = IO; + + auto const tbl = create_random_table( + cycle_dtypes(d_type, num_cols), table_size_bytes{data_size}, data_profile_builder()); + auto const view = tbl->view(); + + cuio_source_sink_pair source_sink(source_type); + cudf::io::json_writer_options write_opts = + cudf::io::json_writer_options::builder(source_sink.make_sink_info(), view).na_rep("null"); + + json_write_common(write_opts, source_sink, data_size, state); +} + +void BM_json_writer_options(nvbench::state& state) +{ + auto const d_type = get_type_or_group({static_cast(data_type::INTEGRAL), + static_cast(data_type::FLOAT), + static_cast(data_type::DECIMAL), + static_cast(data_type::TIMESTAMP), + static_cast(data_type::DURATION), + static_cast(data_type::STRING), + static_cast(data_type::LIST), + static_cast(data_type::STRUCT)}); + + auto const source_type = io_type::HOST_BUFFER; + bool const json_lines = state.get_int64("json_lines"); + bool const include_nulls = state.get_int64("include_nulls"); + auto const rows_per_chunk = state.get_int64("rows_per_chunk"); + + auto const tbl = create_random_table( + cycle_dtypes(d_type, num_cols), table_size_bytes{data_size}, data_profile_builder()); + auto const view = tbl->view(); + + cuio_source_sink_pair source_sink(source_type); + cudf::io::json_writer_options write_opts = + cudf::io::json_writer_options::builder(source_sink.make_sink_info(), view) + .na_rep("null") + .lines(json_lines) + .include_nulls(include_nulls) + .rows_per_chunk(rows_per_chunk); + + json_write_common(write_opts, source_sink, data_size, state); +} + +using io_list = nvbench::enum_type_list; + +NVBENCH_BENCH_TYPES(BM_json_write_io, NVBENCH_TYPE_AXES(io_list)) + .set_name("json_write_io") + .set_type_axes_names({"io"}) + .set_min_samples(4); + +NVBENCH_BENCH(BM_json_writer_options) + .set_name("json_write_options") + .set_min_samples(4) + .add_int64_axis("json_lines", {false, true}) + .add_int64_axis("include_nulls", {false, true}) + .add_int64_power_of_two_axis("rows_per_chunk", nvbench::range(10, 20, 2)); diff --git a/cpp/src/io/json/write_json.cu b/cpp/src/io/json/write_json.cu index a7ae4d3bdd1..b4bcb5548de 100644 --- a/cpp/src/io/json/write_json.cu +++ b/cpp/src/io/json/write_json.cu @@ -240,6 +240,7 @@ std::unique_ptr struct_to_strings(table_view const& strings_columns, rmm::cuda_stream_view stream, rmm::mr::device_memory_resource* mr) { + CUDF_FUNC_RANGE(); CUDF_EXPECTS(column_names.type().id() == type_id::STRING, "Column names must be of type string"); auto const num_columns = strings_columns.num_columns(); CUDF_EXPECTS(num_columns == column_names.size(), @@ -481,6 +482,7 @@ struct column_to_strings_fn { column_iterator column_end, host_span children_names) const { + CUDF_FUNC_RANGE(); auto const num_columns = std::distance(column_begin, column_end); auto column_names = make_column_names_column(children_names, num_columns, stream_); auto column_names_view = column_names->view(); @@ -590,6 +592,7 @@ void write_chunked(data_sink* out_sink, rmm::cuda_stream_view stream, rmm::mr::device_memory_resource* mr) { + CUDF_FUNC_RANGE(); CUDF_EXPECTS(str_column_view.size() > 0, "Unexpected empty strings column."); auto p_str_col_w_nl = cudf::strings::detail::join_strings(str_column_view, @@ -620,6 +623,7 @@ void write_json(data_sink* out_sink, rmm::cuda_stream_view stream, rmm::mr::device_memory_resource* mr) { + CUDF_FUNC_RANGE(); std::vector user_column_names = [&]() { auto const& metadata = options.get_metadata(); if (metadata.has_value() and not metadata->schema_info.empty()) { From 195e2f7b531ce4c5ff80ff5985dd82b7eae43134 Mon Sep 17 00:00:00 2001 From: GALI PREM SAGAR Date: Wed, 1 Mar 2023 15:00:13 -0600 Subject: [PATCH 25/60] Fix docs build to be `pydata-sphinx-theme=0.13.0` compatible (#12874) This PR fixes cudf docs build to be compatible with `pydata-sphinx-theme=0.13.0` by adding an empty `icon_links` entry. Authors: - GALI PREM SAGAR (https://github.com/galipremsagar) Approvers: - Bradley Dice (https://github.com/bdice) - AJ Schmidt (https://github.com/ajschmidt8) URL: https://github.com/rapidsai/cudf/pull/12874 --- docs/cudf/source/conf.py | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/docs/cudf/source/conf.py b/docs/cudf/source/conf.py index 3d92d955263..b97879b0ae4 100644 --- a/docs/cudf/source/conf.py +++ b/docs/cudf/source/conf.py @@ -45,8 +45,8 @@ "myst_nb", ] -jupyter_execute_notebooks = "force" -execution_timeout = 300 +nb_execution_mode = "force" +nb_execution_timeout = 300 copybutton_prompt_text = ">>> " autosummary_generate = True @@ -103,6 +103,8 @@ html_theme_options = { "external_links": [], + # https://github.com/pydata/pydata-sphinx-theme/issues/1220 + "icon_links": [], "github_url": "https://github.com/rapidsai/cudf", "twitter_url": "https://twitter.com/rapidsai", "show_toc_level": 1, From 40e56c94e8450603a169546faad36397f2aef8aa Mon Sep 17 00:00:00 2001 From: Ed Seidl Date: Wed, 1 Mar 2023 15:32:54 -0800 Subject: [PATCH 26/60] Parquet writer column_size() should return a size_t (#12870) Fixes #12867. Bug introduced in #12685. A calculation of total bytes in a column was returned in a 32-bit `size_type` rather than 64-bit `size_t` leading to overflow for tables with many millions of rows. Authors: - Ed Seidl (https://github.com/etseidl) - Vukasin Milovanovic (https://github.com/vuule) - GALI PREM SAGAR (https://github.com/galipremsagar) Approvers: - Vukasin Milovanovic (https://github.com/vuule) - Karthikeyan (https://github.com/karthikeyann) - GALI PREM SAGAR (https://github.com/galipremsagar) URL: https://github.com/rapidsai/cudf/pull/12870 --- cpp/src/io/parquet/writer_impl.cu | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/cpp/src/io/parquet/writer_impl.cu b/cpp/src/io/parquet/writer_impl.cu index 88176ee1901..2c9bff33a14 100644 --- a/cpp/src/io/parquet/writer_impl.cu +++ b/cpp/src/io/parquet/writer_impl.cu @@ -87,7 +87,7 @@ parquet::Compression to_parquet_compression(compression_type compression) } } -size_type column_size(column_view const& column, rmm::cuda_stream_view stream) +size_t column_size(column_view const& column, rmm::cuda_stream_view stream) { if (column.size() == 0) { return 0; } @@ -99,7 +99,7 @@ size_type column_size(column_view const& column, rmm::cuda_stream_view stream) cudf::detail::get_value(scol.offsets(), 0, stream); } else if (column.type().id() == type_id::STRUCT) { auto const scol = structs_column_view(column); - size_type ret = 0; + size_t ret = 0; for (int i = 0; i < scol.num_children(); i++) { ret += column_size(scol.get_sliced_child(i), stream); } From 8747daa66d2b181c5c95d72998d78ab00a5daf60 Mon Sep 17 00:00:00 2001 From: Bradley Dice Date: Thu, 2 Mar 2023 11:31:18 -0800 Subject: [PATCH 27/60] Use python -m pytest for nightly wheel tests (#12871) Nightly wheel tests are failing with this error: ``` + sh -c 'pytest -v -n 8 ./python/cudf/cudf/tests' sh: 1: pytest: not found ``` However, pytest is being installed. The wheel itself is being installed with `python -m pip`, like: ``` python -m pip install --verbose './dist/cudf_cu11-23.4.0.1677647948-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl[test]' ``` Calling `python -m pytest` may resolve this error. For consistency, we should use `python -m` for all executable modules (pip and pytest). Authors: - Bradley Dice (https://github.com/bdice) Approvers: - Jordan Jacobelli (https://github.com/jjacobelli) URL: https://github.com/rapidsai/cudf/pull/12871 --- .github/workflows/pr.yaml | 10 +++++----- .github/workflows/test.yaml | 6 +++--- 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/.github/workflows/pr.yaml b/.github/workflows/pr.yaml index c36c539a102..d02825b73d1 100644 --- a/.github/workflows/pr.yaml +++ b/.github/workflows/pr.yaml @@ -113,8 +113,8 @@ jobs: build_type: pull-request package-name: cudf # Install cupy-cuda11x for arm from a special index url - test-before-arm64: "pip install cupy-cuda11x -f https://pip.cupy.dev/aarch64" - test-unittest: "pytest -v -n 8 ./python/cudf/cudf/tests" + test-before-arm64: "python -m pip install cupy-cuda11x -f https://pip.cupy.dev/aarch64" + test-unittest: "python -m pytest -v -n 8 ./python/cudf/cudf/tests" test-smoketest: "python ./ci/wheel_smoke_test_cudf.py" wheel-build-dask-cudf: needs: wheel-tests-cudf @@ -124,7 +124,7 @@ jobs: build_type: pull-request package-name: dask_cudf package-dir: python/dask_cudf - before-wheel: "RAPIDS_PY_WHEEL_NAME=cudf_cu11 rapids-download-wheels-from-s3 ./local-cudf && pip install --no-deps ./local-cudf/cudf*.whl" + before-wheel: "RAPIDS_PY_WHEEL_NAME=cudf_cu11 rapids-download-wheels-from-s3 ./local-cudf && python -m pip install --no-deps ./local-cudf/cudf*.whl" uses-setup-env-vars: false wheel-tests-dask-cudf: needs: wheel-build-dask-cudf @@ -133,5 +133,5 @@ jobs: with: build_type: pull-request package-name: dask_cudf - test-before: "RAPIDS_PY_WHEEL_NAME=cudf_cu11 rapids-download-wheels-from-s3 ./local-cudf-dep && pip install --no-deps ./local-cudf-dep/cudf*.whl" - test-unittest: "pytest -v -n 8 ./python/dask_cudf/dask_cudf/tests" + test-before: "RAPIDS_PY_WHEEL_NAME=cudf_cu11 rapids-download-wheels-from-s3 ./local-cudf-dep && python -m pip install --no-deps ./local-cudf-dep/cudf*.whl" + test-unittest: "python -m pytest -v -n 8 ./python/dask_cudf/dask_cudf/tests" diff --git a/.github/workflows/test.yaml b/.github/workflows/test.yaml index 5dc04ece919..c808e1475e6 100644 --- a/.github/workflows/test.yaml +++ b/.github/workflows/test.yaml @@ -86,8 +86,8 @@ jobs: date: ${{ inputs.date }} sha: ${{ inputs.sha }} package-name: cudf - test-before-arm64: "pip install cupy-cuda11x -f https://pip.cupy.dev/aarch64" - test-unittest: "pytest -v -n 8 ./python/cudf/cudf/tests" + test-before-arm64: "python -m pip install cupy-cuda11x -f https://pip.cupy.dev/aarch64" + test-unittest: "python -m pytest -v -n 8 ./python/cudf/cudf/tests" wheel-tests-dask-cudf: secrets: inherit uses: rapidsai/shared-action-workflows/.github/workflows/wheels-pure-test.yml@branch-23.04 @@ -97,4 +97,4 @@ jobs: date: ${{ inputs.date }} sha: ${{ inputs.sha }} package-name: dask_cudf - test-unittest: "pytest -v -n 8 ./python/dask_cudf/dask_cudf/tests" + test-unittest: "python -m pytest -v -n 8 ./python/dask_cudf/dask_cudf/tests" From 00c887b143a805b8f967eb91d20d2f651c193e28 Mon Sep 17 00:00:00 2001 From: Vukasin Milovanovic Date: Fri, 3 Mar 2023 16:21:13 -0800 Subject: [PATCH 28/60] Deallocate encoded data in ORC writer immediately after compression (#12770) Reduces total memory footprint over the execution time (but not peak memory usage, which happens during compression). This change helps overall memory usage when writing multiple files in parallel using the same GPU. Authors: - Vukasin Milovanovic (https://github.com/vuule) Approvers: - Nghia Truong (https://github.com/ttnghia) - David Wendt (https://github.com/davidwendt) URL: https://github.com/rapidsai/cudf/pull/12770 --- cpp/src/io/orc/writer_impl.cu | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/cpp/src/io/orc/writer_impl.cu b/cpp/src/io/orc/writer_impl.cu index a6effeefc6c..8d85b001817 100644 --- a/cpp/src/io/orc/writer_impl.cu +++ b/cpp/src/io/orc/writer_impl.cu @@ -2246,6 +2246,10 @@ void writer::impl::write(table_view const& table) enc_data.streams, comp_results, stream); + + // deallocate encoded data as it is not needed anymore + enc_data.data = rmm::device_uvector{0, stream}; + strm_descs.device_to_host(stream); comp_results.device_to_host(stream, true); } From 2689bb698e8e8e11659d599172db8cbc36b76e0f Mon Sep 17 00:00:00 2001 From: Ed Seidl Date: Fri, 3 Mar 2023 17:03:05 -0800 Subject: [PATCH 29/60] Throw an exception if an unsupported page encoding is detected in Parquet reader (#12754) If the Parquet reader comes across a page encoded with an unsupported encoding, the call to decode page data silently fails, leading to either an empty table or unrelated exceptions being thrown. This PR adds code to validate the page encodings after the page headers are decoded. Authors: - Ed Seidl (https://github.com/etseidl) - Vukasin Milovanovic (https://github.com/vuule) Approvers: - GALI PREM SAGAR (https://github.com/galipremsagar) - MithunR (https://github.com/mythrocks) - Nghia Truong (https://github.com/ttnghia) - Vukasin Milovanovic (https://github.com/vuule) URL: https://github.com/rapidsai/cudf/pull/12754 --- cpp/src/io/parquet/page_data.cu | 2 ++ cpp/src/io/parquet/parquet_common.hpp | 5 +++-- cpp/src/io/parquet/reader_impl_preprocess.cu | 18 ++++++++++++++++++ .../tests/data/parquet/delta_encoding.parquet | Bin 0 -> 577 bytes python/cudf/cudf/tests/test_parquet.py | 8 ++++++++ 5 files changed, 31 insertions(+), 2 deletions(-) create mode 100644 python/cudf/cudf/tests/data/parquet/delta_encoding.parquet diff --git a/cpp/src/io/parquet/page_data.cu b/cpp/src/io/parquet/page_data.cu index ee115e7432a..f377e833645 100644 --- a/cpp/src/io/parquet/page_data.cu +++ b/cpp/src/io/parquet/page_data.cu @@ -1167,6 +1167,8 @@ static __device__ bool setupLocalPageInfo(page_state_s* const s, s->dict_bits = 0; s->dict_base = nullptr; s->dict_size = 0; + // NOTE: if additional encodings are supported in the future, modifications must + // be made to is_supported_encoding() in reader_impl_preprocess.cu switch (s->page.encoding) { case Encoding::PLAIN_DICTIONARY: case Encoding::RLE_DICTIONARY: diff --git a/cpp/src/io/parquet/parquet_common.hpp b/cpp/src/io/parquet/parquet_common.hpp index d56f2fb08ca..ab6290c4ed6 100644 --- a/cpp/src/io/parquet/parquet_common.hpp +++ b/cpp/src/io/parquet/parquet_common.hpp @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018-2022, NVIDIA CORPORATION. + * Copyright (c) 2018-2023, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -86,11 +86,12 @@ enum class Encoding : uint8_t { GROUP_VAR_INT = 1, // Deprecated, never used PLAIN_DICTIONARY = 2, RLE = 3, - BIT_PACKED = 4, + BIT_PACKED = 4, // Deprecated by parquet-format in 2013, superseded by RLE DELTA_BINARY_PACKED = 5, DELTA_LENGTH_BYTE_ARRAY = 6, DELTA_BYTE_ARRAY = 7, RLE_DICTIONARY = 8, + BYTE_STREAM_SPLIT = 9, }; /** diff --git a/cpp/src/io/parquet/reader_impl_preprocess.cu b/cpp/src/io/parquet/reader_impl_preprocess.cu index 0f55cd6e400..6b6ad5a2f7d 100644 --- a/cpp/src/io/parquet/reader_impl_preprocess.cu +++ b/cpp/src/io/parquet/reader_impl_preprocess.cu @@ -307,6 +307,18 @@ template return total_pages; } +// see setupLocalPageInfo() in page_data.cu for supported page encodings +constexpr bool is_supported_encoding(Encoding enc) +{ + switch (enc) { + case Encoding::PLAIN: + case Encoding::PLAIN_DICTIONARY: + case Encoding::RLE: + case Encoding::RLE_DICTIONARY: return true; + default: return false; + } +} + /** * @brief Decode the page information from the given column chunks. * @@ -329,6 +341,12 @@ void decode_page_headers(hostdevice_vector& chunks, chunks.host_to_device(stream); gpu::DecodePageHeaders(chunks.device_ptr(), chunks.size(), stream); pages.device_to_host(stream, true); + + // validate page encodings + CUDF_EXPECTS(std::all_of(pages.begin(), + pages.end(), + [](auto const& page) { return is_supported_encoding(page.encoding); }), + "Unsupported page encoding detected"); } /** diff --git a/python/cudf/cudf/tests/data/parquet/delta_encoding.parquet b/python/cudf/cudf/tests/data/parquet/delta_encoding.parquet new file mode 100644 index 0000000000000000000000000000000000000000..e129ced34f3b570ba0ae966277f2111f8f539465 GIT binary patch literal 577 zcmZWnQES^U5LOf~Ghr`-O2}YJ5M5{?6|vPcX($_gE5(d5`nHy#C`xN4+o`Q=lo06q z-u5H+J9bXe=AnAIyYIfc`%cpJ#kYvG$Ufn>U=YEi4c0@b*=l}4?Stor*TfjX2|nV0 z`Hk9z18#HUK}|pb2&qpJaOya@{7}tR`SVAmA55kQMgz<|UD!ShXeH;s8hFgJY2_eC zyib_DH`Zh(cZlL348pD-5-%X_V6-kS!(RG;^Xg2o~x`*)mH+E4e_-=m8E(DczY!` zn>Mz{?*Y_B?o^srUl-Q$F|; Date: Sat, 4 Mar 2023 13:40:36 -0800 Subject: [PATCH 30/60] Use test paths relative to package directory. (#12751) This PR standardizes how our test paths are defined for dask-cudf and custreamz. This was originally a test to investigate a question related to CodeCov, which is answered below. We noticed that some coverage reports in the CI outputs used full paths like `/opt/conda/envs/test/lib/python3.10/site-packages/cudf/__init__.py` while other coverage reports gave relative paths like `cudf/__init__.py`. The net result of this PR is that coverage reports are unchanged, but our CI tests are more correct because they always pull from the installed packages rather than the local packages (`import dask_cudf` is fetching the built conda packages from `site-packages` and not importing from the current path in the cloned repo). Authors: - Bradley Dice (https://github.com/bdice) - GALI PREM SAGAR (https://github.com/galipremsagar) Approvers: - Matthew Roeschke (https://github.com/mroeschke) - Jordan Jacobelli (https://github.com/jjacobelli) URL: https://github.com/rapidsai/cudf/pull/12751 --- ci/test_python_other.sh | 12 ++++++------ python/custreamz/.coveragerc | 3 +++ 2 files changed, 9 insertions(+), 6 deletions(-) create mode 100644 python/custreamz/.coveragerc diff --git a/ci/test_python_other.sh b/ci/test_python_other.sh index 25ed615df84..ab36fbbb5ff 100755 --- a/ci/test_python_other.sh +++ b/ci/test_python_other.sh @@ -17,31 +17,31 @@ trap "EXITCODE=1" ERR set +e rapids-logger "pytest dask_cudf" -pushd python/dask_cudf +pushd python/dask_cudf/dask_cudf pytest \ --cache-clear \ --junitxml="${RAPIDS_TESTS_DIR}/junit-dask-cudf.xml" \ --numprocesses=8 \ --dist=loadscope \ - --cov-config=.coveragerc \ + --cov-config=../.coveragerc \ --cov=dask_cudf \ --cov-report=xml:"${RAPIDS_COVERAGE_DIR}/dask-cudf-coverage.xml" \ --cov-report=term \ - dask_cudf + tests popd rapids-logger "pytest custreamz" -pushd python/custreamz +pushd python/custreamz/custreamz pytest \ --cache-clear \ --junitxml="${RAPIDS_TESTS_DIR}/junit-custreamz.xml" \ --numprocesses=8 \ --dist=loadscope \ - --cov-config=.coveragerc \ + --cov-config=../.coveragerc \ --cov=custreamz \ --cov-report=xml:"${RAPIDS_COVERAGE_DIR}/custreamz-coverage.xml" \ --cov-report=term \ - custreamz + tests popd rapids-logger "Test script exiting with value: $EXITCODE" diff --git a/python/custreamz/.coveragerc b/python/custreamz/.coveragerc new file mode 100644 index 00000000000..26bed7816ed --- /dev/null +++ b/python/custreamz/.coveragerc @@ -0,0 +1,3 @@ +# Configuration file for Python coverage tests +[run] +source = custreamz From 77b5014200ad6415e06e43216db477cafeb07c63 Mon Sep 17 00:00:00 2001 From: Vyas Ramasubramani Date: Mon, 6 Mar 2023 12:39:17 -0500 Subject: [PATCH 31/60] Expect cupy to now support bool arrays for dlpack. (#12883) The most recent versions of cupy now support bool arrays in dlpack (see [the release notes](https://github.com/cupy/cupy/releases/tag/v11.6.0) and [the PR adding the feature](https://github.com/cupy/cupy/pull/7376)). Our DataFrame protocol tests were expecting a failure. Authors: - Vyas Ramasubramani (https://github.com/vyasr) Approvers: - Bradley Dice (https://github.com/bdice) - Lawrence Mitchell (https://github.com/wence-) URL: https://github.com/rapidsai/cudf/pull/12883 --- python/cudf/cudf/tests/test_df_protocol.py | 15 ++++++--------- 1 file changed, 6 insertions(+), 9 deletions(-) diff --git a/python/cudf/cudf/tests/test_df_protocol.py b/python/cudf/cudf/tests/test_df_protocol.py index 7dbca90ab03..8a53c77da66 100644 --- a/python/cudf/cudf/tests/test_df_protocol.py +++ b/python/cudf/cudf/tests/test_df_protocol.py @@ -39,15 +39,12 @@ def assert_buffer_equal(buffer_and_dtype: Tuple[_CuDFBuffer, Any], cudfcol): cudfcol.apply_boolean_mask(non_null_idxs), ) - if dtype[0] != _DtypeKind.BOOL: - array_from_dlpack = cp.from_dlpack(buf.__dlpack__()).get() - col_array = cp.asarray(cudfcol.data_array_view(mode="read")).get() - assert_eq( - array_from_dlpack[non_null_idxs.to_numpy()].flatten(), - col_array[non_null_idxs.to_numpy()].flatten(), - ) - else: - pytest.raises(TypeError, buf.__dlpack__) + array_from_dlpack = cp.from_dlpack(buf.__dlpack__()).get() + col_array = cp.asarray(cudfcol.data_array_view(mode="read")).get() + assert_eq( + array_from_dlpack[non_null_idxs.to_numpy()].flatten(), + col_array[non_null_idxs.to_numpy()].flatten(), + ) def assert_column_equal(col: _CuDFColumn, cudfcol): From b6d9fd1f26ccda9b3043a8967621bae234187c37 Mon Sep 17 00:00:00 2001 From: Bradley Dice Date: Mon, 6 Mar 2023 10:04:08 -0800 Subject: [PATCH 32/60] Split C++ and Python build dependencies into separate lists. (#12724) This PR splits the build dependencies in `dependencies.yaml` into C++, Python, and shared (`all`) dependency lists. This makes it easier to understand the dependency tree for libcudf and cudf Python. Authors: - Bradley Dice (https://github.com/bdice) Approvers: - Ray Douglass (https://github.com/raydouglass) - AJ Schmidt (https://github.com/ajschmidt8) URL: https://github.com/rapidsai/cudf/pull/12724 --- .../all_cuda-118_arch-x86_64.yaml | 1 + dependencies.yaml | 35 +++++++++++++------ 2 files changed, 26 insertions(+), 10 deletions(-) diff --git a/conda/environments/all_cuda-118_arch-x86_64.yaml b/conda/environments/all_cuda-118_arch-x86_64.yaml index 44d6be65574..00aa1e2bc0f 100644 --- a/conda/environments/all_cuda-118_arch-x86_64.yaml +++ b/conda/environments/all_cuda-118_arch-x86_64.yaml @@ -33,6 +33,7 @@ dependencies: - ipython - libarrow=10 - librdkafka=1.7.0 +- librmm=23.04.* - mimesis>=4.1.0 - moto>=4.0.8 - myst-nb diff --git a/dependencies.yaml b/dependencies.yaml index ba6d240e069..2c359d58378 100644 --- a/dependencies.yaml +++ b/dependencies.yaml @@ -6,7 +6,9 @@ files: cuda: ["11.8"] arch: [x86_64] includes: - - build + - build_all + - build_cpp + - build_python - cudatoolkit - develop - docs @@ -30,7 +32,7 @@ files: test_java: output: none includes: - - build + - build_all - cudatoolkit - test_java test_notebooks: @@ -57,25 +59,18 @@ channels: - conda-forge - nvidia dependencies: - build: + build_all: common: - output_types: [conda, requirements] packages: - &cmake_ver cmake>=3.23.1,!=3.25.0 - - cuda-python>=11.7.1,<12.0 - - cython>=0.29,<0.30 - dlpack>=0.5,<0.6.0a0 - ninja - - pyarrow=10 - - rmm=23.04.* - - scikit-build>=0.13.1 - output_types: conda packages: - libarrow=10 - c-compiler - cxx-compiler - - librdkafka=1.7.0 - - protobuf=4.21 specific: - output_types: conda matrices: @@ -101,6 +96,26 @@ dependencies: cuda: "11.8" packages: - nvcc_linux-aarch64=11.8 + build_cpp: + common: + - output_types: [conda, requirements] + packages: + - librmm=23.04.* + - output_types: conda + packages: + - librdkafka=1.7.0 + build_python: + common: + - output_types: [conda, requirements] + packages: + - cuda-python>=11.7.1,<12.0 + - cython>=0.29,<0.30 + - pyarrow=10 + - rmm=23.04.* + - scikit-build>=0.13.1 + - output_types: conda + packages: + - protobuf=4.21 cudatoolkit: specific: - output_types: conda From 39398e4dedb0b2537e07d4aea1f52ac23ea8108b Mon Sep 17 00:00:00 2001 From: GALI PREM SAGAR Date: Mon, 6 Mar 2023 13:34:21 -0600 Subject: [PATCH 33/60] Deprecate `datetime_is_numeric` from `describe` (#12818) `datetime_is_numeric` is non-functional parameter in `cudf` and is removed in `pandas 2.0`, hence deprecating this param for removal. Authors: - GALI PREM SAGAR (https://github.com/galipremsagar) Approvers: - Bradley Dice (https://github.com/bdice) URL: https://github.com/rapidsai/cudf/pull/12818 --- .../developer_guide/contributing_guide.md | 8 ++++++++ python/cudf/cudf/core/dataframe.py | 12 +++++++++++- python/cudf/cudf/core/series.py | 8 ++++++++ python/cudf/cudf/tests/test_dataframe.py | 18 ++++++++++++------ python/cudf/cudf/tests/test_series.py | 12 ++++++++---- python/cudf/cudf/utils/docutils.py | 7 ++++++- 6 files changed, 53 insertions(+), 12 deletions(-) diff --git a/docs/cudf/source/developer_guide/contributing_guide.md b/docs/cudf/source/developer_guide/contributing_guide.md index b5ea9519842..3244959b14c 100644 --- a/docs/cudf/source/developer_guide/contributing_guide.md +++ b/docs/cudf/source/developer_guide/contributing_guide.md @@ -67,6 +67,14 @@ Deprecations should be signaled using a `FutureWarning` **not a `DeprecationWarn `DeprecationWarning` is hidden by default except in code run in the `__main__` module. ``` +Deprecations should also be specified in the respective public API docstring using a +`deprecated` admonition: + +``` +.. deprecated:: 23.08 + `foo` is deprecated and will be removed in a future version of cudf. +``` + ## `pandas` compatibility Maintaining compatibility with the [pandas API](https://pandas.pydata.org/docs/reference/index.html) is a primary goal of cuDF. diff --git a/python/cudf/cudf/core/dataframe.py b/python/cudf/cudf/core/dataframe.py index d43621d3d36..fd6e9e2687d 100644 --- a/python/cudf/cudf/core/dataframe.py +++ b/python/cudf/cudf/core/dataframe.py @@ -4937,6 +4937,13 @@ def describe( default_include = [np.number] if datetime_is_numeric: default_include.append("datetime") + else: + warnings.warn( + "`datetime_is_numeric` is deprecated. Specify " + "`datetime_is_numeric=True` to silence this " + "warning and adopt the future behavior now.", + FutureWarning, + ) data_to_describe = self.select_dtypes(include=default_include) if data_to_describe._num_columns == 0: data_to_describe = self @@ -4955,7 +4962,10 @@ def describe( raise ValueError("No data of included types.") describe_series_list = [ - data_to_describe[col].describe(percentiles=percentiles) + data_to_describe[col].describe( + percentiles=percentiles, + datetime_is_numeric=datetime_is_numeric, + ) for col in data_to_describe._column_names ] if len(describe_series_list) == 1: diff --git a/python/cudf/cudf/core/series.py b/python/cudf/cudf/core/series.py index 60655c5a6f9..79927c60a85 100644 --- a/python/cudf/cudf/core/series.py +++ b/python/cudf/cudf/core/series.py @@ -6,6 +6,7 @@ import inspect import pickle import textwrap +import warnings from collections import abc from shutil import get_terminal_size from typing import Any, Dict, MutableMapping, Optional, Set, Tuple, Union @@ -3111,6 +3112,13 @@ def describe( ): """{docstring}""" + if not datetime_is_numeric: + warnings.warn( + "`datetime_is_numeric` is deprecated and will be removed in " + "a future release. Specify `datetime_is_numeric=True` to " + "silence this warning and adopt the future behavior now.", + FutureWarning, + ) if percentiles is not None: if not all(0 <= x <= 1 for x in percentiles): raise ValueError( diff --git a/python/cudf/cudf/tests/test_dataframe.py b/python/cudf/cudf/tests/test_dataframe.py index 13f312f6f0c..7ddfa3a7f48 100644 --- a/python/cudf/cudf/tests/test_dataframe.py +++ b/python/cudf/cudf/tests/test_dataframe.py @@ -3656,7 +3656,8 @@ def test_dataframe_describe_exclude(): df["x"] = df.x.astype("int64") df["y"] = np.random.normal(10, 1, data_length) pdf = df.to_pandas() - gdf_results = df.describe(exclude=["float"]) + with pytest.warns(FutureWarning): + gdf_results = df.describe(exclude=["float"]) pdf_results = pdf.describe(exclude=["float"]) assert_eq(gdf_results, pdf_results) @@ -3671,7 +3672,8 @@ def test_dataframe_describe_include(): df["x"] = df.x.astype("int64") df["y"] = np.random.normal(10, 1, data_length) pdf = df.to_pandas() - gdf_results = df.describe(include=["int"]) + with pytest.warns(FutureWarning): + gdf_results = df.describe(include=["int"]) pdf_results = pdf.describe(include=["int"]) assert_eq(gdf_results, pdf_results) @@ -3685,7 +3687,8 @@ def test_dataframe_describe_default(): df["x"] = np.random.normal(10, 1, data_length) df["y"] = np.random.normal(10, 1, data_length) pdf = df.to_pandas() - gdf_results = df.describe() + with pytest.warns(FutureWarning): + gdf_results = df.describe() pdf_results = pdf.describe() assert_eq(pdf_results, gdf_results) @@ -3702,7 +3705,8 @@ def test_series_describe_include_all(): df["animal"] = np.random.choice(["dog", "cat", "bird"], data_length) pdf = df.to_pandas() - gdf_results = df.describe(include="all") + with pytest.warns(FutureWarning): + gdf_results = df.describe(include="all") pdf_results = pdf.describe(include="all") assert_eq(gdf_results[["x", "y"]], pdf_results[["x", "y"]]) @@ -3723,7 +3727,8 @@ def test_dataframe_describe_percentiles(): df["x"] = np.random.normal(10, 1, data_length) df["y"] = np.random.normal(10, 1, data_length) pdf = df.to_pandas() - gdf_results = df.describe(percentiles=sample_percentiles) + with pytest.warns(FutureWarning): + gdf_results = df.describe(percentiles=sample_percentiles) pdf_results = pdf.describe(percentiles=sample_percentiles) assert_eq(pdf_results, gdf_results) @@ -4050,7 +4055,8 @@ def test_empty_dataframe_describe(): gdf = cudf.from_pandas(pdf) expected = pdf.describe() - actual = gdf.describe() + with pytest.warns(FutureWarning): + actual = gdf.describe() assert_eq(expected, actual) diff --git a/python/cudf/cudf/tests/test_series.py b/python/cudf/cudf/tests/test_series.py index b3c7c9ac9bb..eb05468d923 100644 --- a/python/cudf/cudf/tests/test_series.py +++ b/python/cudf/cudf/tests/test_series.py @@ -408,7 +408,8 @@ def test_series_size(data): def test_series_describe_numeric(dtype): ps = pd.Series([0, 1, 2, 3, 1, 2, 3], dtype=dtype) gs = cudf.from_pandas(ps) - actual = gs.describe() + with pytest.warns(FutureWarning): + actual = gs.describe() expected = ps.describe() assert_eq(expected, actual, check_dtype=True) @@ -426,7 +427,8 @@ def test_series_describe_datetime(dtype): # Treating datetimes as categoricals is deprecated in pandas and will # be removed in future. Future behavior is treating datetime as numeric. expected = ps.describe(datetime_is_numeric=True) - actual = gs.describe() + with pytest.warns(FutureWarning): + actual = gs.describe() assert_eq(expected.astype("str"), actual) @@ -437,7 +439,8 @@ def test_series_describe_timedelta(dtype): gs = cudf.from_pandas(ps) expected = ps.describe() - actual = gs.describe() + with pytest.warns(FutureWarning): + actual = gs.describe() assert_eq(actual, expected.astype("str")) @@ -462,7 +465,8 @@ def test_series_describe_other_types(ps): gs = cudf.from_pandas(ps) expected = ps.describe() - actual = gs.describe() + with pytest.warns(FutureWarning): + actual = gs.describe() if len(ps) == 0: assert_eq(expected.fillna("a").astype("str"), actual.fillna("a")) diff --git a/python/cudf/cudf/utils/docutils.py b/python/cudf/cudf/utils/docutils.py index 72ebae05b6f..5a7b8bae980 100644 --- a/python/cudf/cudf/utils/docutils.py +++ b/python/cudf/cudf/utils/docutils.py @@ -1,4 +1,4 @@ -# Copyright (c) 2018-2022, NVIDIA CORPORATION. +# Copyright (c) 2018-2023, NVIDIA CORPORATION. """ Helper functions for parameterized docstring @@ -130,6 +130,11 @@ def wrapper(func): For DataFrame input, this also controls whether datetime columns are included by default. + .. deprecated:: 23.04 + + `datetime_is_numeric` is deprecated and will be removed in + a future version of cudf. + Returns ------- output_frame : Series or DataFrame From 7dade51f8636f00153b0c0f190dbbb352c5b309b Mon Sep 17 00:00:00 2001 From: Vyas Ramasubramani Date: Mon, 6 Mar 2023 14:45:55 -0500 Subject: [PATCH 34/60] Migrate as much as possible to pyproject.toml (#12850) I migrated as much build logic as possible to pyproject.toml for cudf, dask_cudf, and custreamz. A couple of notes: - I didn't move cudf_kafka because it is currently still using a pure setuptools build with Cython files. We may as well do the pyproject.toml migration there after we have also done the scikit-build conversion since there's a lot of logic that can't be removed from setup.py until then. I'm also not entirely confident that everything will work as expected if we move data to pyproject.toml without switching to scikit-build based on issues like the ones mentioned in [this SO post about Cython and project data stored in pyproject.toml](https://stackoverflow.com/questions/73800736/pyproject-toml-and-cython-extension-module). - I could get rid of setup.py entirely for custreamz at this point, but I expect that we'll run into a host of additional issues when we do eventually move to requiring `pip install X` instead of `python setup.py X` across RAPIDS and I'd rather deal with all of those at the same time with our main packages rather than using an esoteric package like custreamz as the test case. Authors: - Vyas Ramasubramani (https://github.com/vyasr) - GALI PREM SAGAR (https://github.com/galipremsagar) Approvers: - Ray Douglass (https://github.com/raydouglass) - GALI PREM SAGAR (https://github.com/galipremsagar) URL: https://github.com/rapidsai/cudf/pull/12850 --- ci/release/apply_wheel_modifications.sh | 25 +++++------ ci/release/update-version.sh | 15 +++---- python/cudf/MANIFEST.in | 16 +++++++ python/cudf/README.md | 1 + python/cudf/pyproject.toml | 60 +++++++++++++++++++++++++ python/cudf/setup.py | 56 ----------------------- python/custreamz/LICENSE | 1 + python/custreamz/pyproject.toml | 47 ++++++++++++++++++- python/custreamz/setup.py | 30 +------------ python/dask_cudf/pyproject.toml | 50 ++++++++++++++++++++- python/dask_cudf/setup.cfg | 5 --- python/dask_cudf/setup.py | 45 +++---------------- 12 files changed, 198 insertions(+), 153 deletions(-) create mode 100644 python/cudf/MANIFEST.in create mode 120000 python/cudf/README.md create mode 120000 python/custreamz/LICENSE delete mode 100644 python/dask_cudf/setup.cfg diff --git a/ci/release/apply_wheel_modifications.sh b/ci/release/apply_wheel_modifications.sh index e017b24be6e..9d9758f1f15 100755 --- a/ci/release/apply_wheel_modifications.sh +++ b/ci/release/apply_wheel_modifications.sh @@ -12,22 +12,19 @@ sed -i "s/__version__ = .*/__version__ = \"${VERSION}\"/g" python/dask_cudf/dask sed -i "s/__version__ = .*/__version__ = \"${VERSION}\"/g" python/cudf_kafka/cudf_kafka/__init__.py sed -i "s/__version__ = .*/__version__ = \"${VERSION}\"/g" python/custreamz/custreamz/__init__.py -# setup.py versions -sed -i "s/version=.*,/version=\"${VERSION}\",/g" python/cudf/setup.py -sed -i "s/version=.*,/version=\"${VERSION}\",/g" python/dask_cudf/setup.py -sed -i "s/version=.*,/version=\"${VERSION}\",/g" python/cudf_kafka/setup.py -sed -i "s/version=.*,/version=\"${VERSION}\",/g" python/custreamz/setup.py - -# cudf setup.py cuda suffixes -sed -i "s/name=\"cudf\"/name=\"cudf${CUDA_SUFFIX}\"/g" python/cudf/setup.py -sed -i "s/rmm/rmm${CUDA_SUFFIX}/g" python/cudf/setup.py -sed -i "s/ptxcompiler/ptxcompiler${CUDA_SUFFIX}/g" python/cudf/setup.py -sed -i "s/cubinlinker/cubinlinker${CUDA_SUFFIX}/g" python/cudf/setup.py +# pyproject.toml versions +sed -i "s/^version = .*/version = \"${VERSION}\"/g" python/cudf/pyproject.toml +sed -i "s/^version = .*/version = \"${VERSION}\"/g" python/dask_cudf/pyproject.toml +sed -i "s/^version = .*/version = \"${VERSION}\"/g" python/cudf_kafka/pyproject.toml +sed -i "s/^version = .*/version = \"${VERSION}\"/g" python/custreamz/pyproject.toml # cudf pyproject.toml cuda suffixes +sed -i "s/^name = \"cudf\"/name = \"cudf${CUDA_SUFFIX}\"/g" python/cudf/pyproject.toml sed -i "s/rmm/rmm${CUDA_SUFFIX}/g" python/cudf/pyproject.toml +sed -i "s/ptxcompiler/ptxcompiler${CUDA_SUFFIX}/g" python/cudf/pyproject.toml +sed -i "s/cubinlinker/cubinlinker${CUDA_SUFFIX}/g" python/cudf/pyproject.toml -# dask_cudf setup.py cuda suffixes -sed -i "s/name=\"dask-cudf\"/name=\"dask-cudf${CUDA_SUFFIX}\"/g" python/dask_cudf/setup.py +# dask_cudf pyproject.toml cuda suffixes +sed -i "s/^name = \"dask_cudf\"/name = \"dask_cudf${CUDA_SUFFIX}\"/g" python/dask_cudf/pyproject.toml # Need to provide the == to avoid modifying the URL -sed -i "s/\"cudf==/\"cudf${CUDA_SUFFIX}==/g" python/dask_cudf/setup.py +sed -i "s/\"cudf==/\"cudf${CUDA_SUFFIX}==/g" python/dask_cudf/pyproject.toml diff --git a/ci/release/update-version.sh b/ci/release/update-version.sh index 831b91bb2a6..e5c9ba0569f 100755 --- a/ci/release/update-version.sh +++ b/ci/release/update-version.sh @@ -49,11 +49,11 @@ sed_runner "s/__version__ = .*/__version__ = \"${NEXT_FULL_TAG}\"/g" python/dask sed_runner "s/__version__ = .*/__version__ = \"${NEXT_FULL_TAG}\"/g" python/cudf_kafka/cudf_kafka/__init__.py sed_runner "s/__version__ = .*/__version__ = \"${NEXT_FULL_TAG}\"/g" python/custreamz/custreamz/__init__.py -# Python setup.py updates -sed_runner "s/version=.*,/version=\"${NEXT_FULL_TAG}\",/g" python/cudf/setup.py -sed_runner "s/version=.*,/version=\"${NEXT_FULL_TAG}\",/g" python/dask_cudf/setup.py -sed_runner "s/version=.*,/version=\"${NEXT_FULL_TAG}\",/g" python/cudf_kafka/setup.py -sed_runner "s/version=.*,/version=\"${NEXT_FULL_TAG}\",/g" python/custreamz/setup.py +# Python pyproject.toml updates +sed_runner "s/^version = .*/version = \"${NEXT_FULL_TAG}\"/g" python/cudf/pyproject.toml +sed_runner "s/^version = .*/version = \"${NEXT_FULL_TAG}\"/g" python/dask_cudf/pyproject.toml +sed_runner "s/^version = .*/version = \"${NEXT_FULL_TAG}\"/g" python/cudf_kafka/pyproject.toml +sed_runner "s/^version = .*/version = \"${NEXT_FULL_TAG}\"/g" python/custreamz/pyproject.toml # rapids-cmake version sed_runner 's/'"branch-.*\/RAPIDS.cmake"'/'"branch-${NEXT_SHORT_TAG}\/RAPIDS.cmake"'/g' fetch_rapids.cmake @@ -89,12 +89,9 @@ sed_runner "s/CUDF_TAG branch-${CURRENT_SHORT_TAG}/CUDF_TAG branch-${NEXT_SHORT_ # Need to distutils-normalize the original version NEXT_SHORT_TAG_PEP440=$(python -c "from setuptools.extern import packaging; print(packaging.version.Version('${NEXT_SHORT_TAG}'))") -# Dependency versions in setup.py -sed_runner "s/rmm==.*\",/rmm==${NEXT_SHORT_TAG_PEP440}.*\",/g" python/cudf/setup.py -sed_runner "s/cudf==.*\",/cudf==${NEXT_SHORT_TAG_PEP440}.*\",/g" python/dask_cudf/setup.py - # Dependency versions in pyproject.toml sed_runner "s/rmm==.*\",/rmm==${NEXT_SHORT_TAG_PEP440}.*\",/g" python/cudf/pyproject.toml +sed_runner "s/cudf==.*\",/cudf==${NEXT_SHORT_TAG_PEP440}.*\",/g" python/dask_cudf/pyproject.toml for FILE in .github/workflows/*.yaml; do sed_runner "/shared-action-workflows/ s/@.*/@branch-${NEXT_SHORT_TAG}/g" "${FILE}" diff --git a/python/cudf/MANIFEST.in b/python/cudf/MANIFEST.in new file mode 100644 index 00000000000..4d3155158f8 --- /dev/null +++ b/python/cudf/MANIFEST.in @@ -0,0 +1,16 @@ +# Cython files +recursive-include cudf *.pxd +recursive-include cudf *.pyx + +# Typing files +recursive-include cudf *.pyi + +# C++ files +recursive-include cudf *.hpp +recursive-include udf_cpp *.hpp +recursive-include udf_cpp *.cuh + +# Build files. Don't use a recursive include on '.' in case the repo is dirty +include . CMakeLists.txt +recursive-include cudf CMakeLists.txt +recursive-include cmake * diff --git a/python/cudf/README.md b/python/cudf/README.md new file mode 120000 index 00000000000..fe840054137 --- /dev/null +++ b/python/cudf/README.md @@ -0,0 +1 @@ +../../README.md \ No newline at end of file diff --git a/python/cudf/pyproject.toml b/python/cudf/pyproject.toml index 305e8822030..0dc719a0846 100644 --- a/python/cudf/pyproject.toml +++ b/python/cudf/pyproject.toml @@ -16,6 +16,66 @@ requires = [ "rmm==23.4.*", ] +[project] +name = "cudf" +version = "23.04.00" +description = "cuDF - GPU Dataframe" +readme = { file = "README.md", content-type = "text/markdown" } +authors = [ + { name = "NVIDIA Corporation" }, +] +license = { text = "Apache 2.0" } +requires-python = ">=3.8" +dependencies = [ + "cachetools", + "cuda-python>=11.7.1,<12.0", + "fsspec>=0.6.0", + "numba>=0.56.2", + "numpy", + "nvtx>=0.2.1", + "packaging", + "pandas>=1.0,<1.6.0dev0", + "protobuf==4.21", + "typing_extensions", + # Allow floating minor versions for Arrow. + "pyarrow==10", + "rmm==23.4.*", + "ptxcompiler", + "cubinlinker", + "cupy-cuda11x", +] +classifiers = [ + "Intended Audience :: Developers", + "Topic :: Database", + "Topic :: Scientific/Engineering", + "License :: OSI Approved :: Apache Software License", + "Programming Language :: Python", + "Programming Language :: Python :: 3.8", + "Programming Language :: Python :: 3.10", +] + +[project.optional-dependencies] +test = [ + "pytest", + "pytest-benchmark", + "pytest-xdist", + "hypothesis", + "mimesis>=4.1.0", + "fastavro>=0.22.9", + "python-snappy>=0.6.0", + "pyorc", + "msgpack", + "transformers==4.24.0", + "tzdata", +] + +[project.urls] +Homepage = "https://github.com/rapidsai/cudf" +Documentation = "https://docs.rapids.ai/api/cudf/stable/" + +[tool.setuptools] +license-files = ["LICENSE"] + [tool.isort] line_length = 79 multi_line_output = 3 diff --git a/python/cudf/setup.py b/python/cudf/setup.py index 0150c4fe715..8a7ebf574fe 100644 --- a/python/cudf/setup.py +++ b/python/cudf/setup.py @@ -3,64 +3,8 @@ from setuptools import find_packages from skbuild import setup -install_requires = [ - "cachetools", - "cuda-python>=11.7.1,<12.0", - "fsspec>=0.6.0", - "numba>=0.56.2", - "numpy", - "nvtx>=0.2.1", - "packaging", - "pandas>=1.0,<1.6.0dev0", - "protobuf==4.21", - "typing_extensions", - # Allow floating minor versions for Arrow. - "pyarrow==10", - "rmm==23.4.*", - "ptxcompiler", - "cubinlinker", - "cupy-cuda11x", -] - -extras_require = { - "test": [ - "pytest", - "pytest-benchmark", - "pytest-xdist", - "hypothesis", - "mimesis>=4.1.0", - "fastavro>=0.22.9", - "python-snappy>=0.6.0", - "pyorc", - "msgpack", - "transformers==4.24.0", - "tzdata", - ] -} - setup( - name="cudf", - version="23.04.00", - description="cuDF - GPU Dataframe", - url="https://github.com/rapidsai/cudf", - author="NVIDIA Corporation", - license="Apache 2.0", - classifiers=[ - "Intended Audience :: Developers", - "Topic :: Database", - "Topic :: Scientific/Engineering", - "License :: OSI Approved :: Apache Software License", - "Programming Language :: Python", - "Programming Language :: Python :: 3.8", - "Programming Language :: Python :: 3.9", - "Programming Language :: Python :: 3.10", - ], include_package_data=True, packages=find_packages(include=["cudf", "cudf.*"]), - package_data={ - key: ["*.pxd"] for key in find_packages(include=["cudf._lib*"]) - }, - install_requires=install_requires, - extras_require=extras_require, zip_safe=False, ) diff --git a/python/custreamz/LICENSE b/python/custreamz/LICENSE new file mode 120000 index 00000000000..30cff7403da --- /dev/null +++ b/python/custreamz/LICENSE @@ -0,0 +1 @@ +../../LICENSE \ No newline at end of file diff --git a/python/custreamz/pyproject.toml b/python/custreamz/pyproject.toml index d5c41945482..315621fa3c1 100644 --- a/python/custreamz/pyproject.toml +++ b/python/custreamz/pyproject.toml @@ -1,12 +1,57 @@ # Copyright (c) 2021-2022, NVIDIA CORPORATION. [build-system] - +build-backend = "setuptools.build_meta" requires = [ "wheel", "setuptools", ] +[project] +name = "custreamz" +version = "23.04.00" +description = "cuStreamz - GPU Accelerated Streaming" +readme = { file = "README.md", content-type = "text/markdown" } +authors = [ + { name = "NVIDIA Corporation" }, +] +license = { text = "Apache 2.0" } +requires-python = ">=3.8" +dependencies = [ + "cudf", + "cudf_kafka", +] +classifiers = [ + "Intended Audience :: Developers", + "Topic :: Streaming", + "Topic :: Scientific/Engineering", + "Topic :: Apache Kafka", + "License :: OSI Approved :: Apache Software License", + "Programming Language :: Python", + "Programming Language :: Python :: 3.8", + "Programming Language :: Python :: 3.9", + "Programming Language :: Python :: 3.10", +] + +[project.optional-dependencies] +test = [ + "pytest", + "pytest-xdist", +] + +[project.urls] +Homepage = "https://github.com/rapidsai/cudf" + +[tool.setuptools] +license-files = ["LICENSE"] +zip-safe = false + +[tools.setuptools.packages.find] +include = [ + "custreamz", + "custreamz.*", +] + [tool.isort] line_length = 79 multi_line_output = 3 diff --git a/python/custreamz/setup.py b/python/custreamz/setup.py index 65a7aac6395..2fa45ac8087 100644 --- a/python/custreamz/setup.py +++ b/python/custreamz/setup.py @@ -1,31 +1,5 @@ # Copyright (c) 2020-2023, NVIDIA CORPORATION. -from setuptools import find_packages, setup +from setuptools import setup -install_requires = ["cudf_kafka", "cudf"] - -extras_require = {"test": ["pytest", "pytest-xdist"]} - -setup( - name="custreamz", - version="23.04.00", - description="cuStreamz - GPU Accelerated Streaming", - url="https://github.com/rapidsai/cudf", - author="NVIDIA Corporation", - license="Apache 2.0", - classifiers=[ - "Intended Audience :: Developers", - "Topic :: Streaming", - "Topic :: Scientific/Engineering", - "Topic :: Apache Kafka", - "License :: OSI Approved :: Apache Software License", - "Programming Language :: Python", - "Programming Language :: Python :: 3.8", - "Programming Language :: Python :: 3.9", - "Programming Language :: Python :: 3.10", - ], - packages=find_packages(include=["custreamz", "custreamz.*"]), - install_requires=install_requires, - extras_require=extras_require, - zip_safe=False, -) +setup() diff --git a/python/dask_cudf/pyproject.toml b/python/dask_cudf/pyproject.toml index 8cf823d4291..07b0edb6008 100644 --- a/python/dask_cudf/pyproject.toml +++ b/python/dask_cudf/pyproject.toml @@ -1,12 +1,58 @@ -# Copyright (c) 2021-2022, NVIDIA CORPORATION. +# Copyright (c) 2021-2023, NVIDIA CORPORATION. [build-system] - +build-backend = "setuptools.build_meta" requires = [ "wheel", "setuptools", ] +[project] +name = "dask_cudf" +version = "23.04.00" +description = "Utilities for Dask and cuDF interactions" +readme = { file = "README.md", content-type = "text/markdown" } +authors = [ + { name = "NVIDIA Corporation" }, +] +license = { text = "Apache 2.0" } +requires-python = ">=3.8" +dependencies = [ + "dask>=2023.1.1", + "distributed>=2023.1.1", + "fsspec>=0.6.0", + "numpy", + "pandas>=1.0,<1.6.0dev0", + "cudf==23.4.*", + "cupy-cuda11x", +] +classifiers = [ + "Intended Audience :: Developers", + "Topic :: Database", + "Topic :: Scientific/Engineering", + "License :: OSI Approved :: Apache Software License", + "Programming Language :: Python", + "Programming Language :: Python :: 3.8", + "Programming Language :: Python :: 3.9", + "Programming Language :: Python :: 3.10", +] +dynamic = ["entry-points"] + +[project.optional-dependencies] +test = [ + "numpy", + "pandas>=1.0,<1.6.0dev0", + "pytest", + "pytest-xdist", + "numba>=0.56.2", +] + +[project.urls] +Homepage = "https://github.com/rapidsai/cudf" + +[tool.setuptools] +license-files = ["LICENSE"] + [tool.isort] line_length = 79 multi_line_output = 3 diff --git a/python/dask_cudf/setup.cfg b/python/dask_cudf/setup.cfg deleted file mode 100644 index 8139b3c7dc6..00000000000 --- a/python/dask_cudf/setup.cfg +++ /dev/null @@ -1,5 +0,0 @@ -# Copyright (c) 2020-2023, NVIDIA CORPORATION. - -[options.entry_points] -dask.dataframe.backends = - cudf = dask_cudf.backends:CudfBackendEntrypoint diff --git a/python/dask_cudf/setup.py b/python/dask_cudf/setup.py index 8611f2379f7..3fa0f257834 100644 --- a/python/dask_cudf/setup.py +++ b/python/dask_cudf/setup.py @@ -2,44 +2,13 @@ from setuptools import find_packages, setup -install_requires = [ - "dask>=2023.1.1", - "distributed>=2023.1.1", - "fsspec>=0.6.0", - "numpy", - "pandas>=1.0,<1.6.0dev0", - "cudf==23.4.*", - "cupy-cuda11x", -] - -extras_require = { - "test": [ - "numpy", - "pandas>=1.0,<1.6.0dev0", - "pytest", - "pytest-xdist", - "numba>=0.56.2", - ] -} - setup( - name="dask-cudf", - version="23.04.00", - description="Utilities for Dask and cuDF interactions", - url="https://github.com/rapidsai/cudf", - author="NVIDIA Corporation", - license="Apache 2.0", - classifiers=[ - "Intended Audience :: Developers", - "Topic :: Database", - "Topic :: Scientific/Engineering", - "License :: OSI Approved :: Apache Software License", - "Programming Language :: Python", - "Programming Language :: Python :: 3.8", - "Programming Language :: Python :: 3.9", - "Programming Language :: Python :: 3.10", - ], + include_package_data=True, packages=find_packages(exclude=["tests", "tests.*"]), - install_requires=install_requires, - extras_require=extras_require, + entry_points={ + "dask.dataframe.backends": [ + "cudf = dask_cudf.backends:CudfBackendEntrypoint", + ] + }, + zip_safe=False, ) From 618194d944199bff964b2dbca4947c6708f42d8c Mon Sep 17 00:00:00 2001 From: GALI PREM SAGAR Date: Mon, 6 Mar 2023 14:52:11 -0600 Subject: [PATCH 35/60] Deprecate `names` & `dtype` in `Index.copy` (#12825) This PR deprecates `dtype` in `Index & MultiIndex`.`copy`, and `names` in `Index.copy`. Authors: - GALI PREM SAGAR (https://github.com/galipremsagar) Approvers: - Vyas Ramasubramani (https://github.com/vyasr) URL: https://github.com/rapidsai/cudf/pull/12825 --- python/cudf/cudf/core/index.py | 38 ++++++++++++++++++++++++++++- python/cudf/cudf/core/multiindex.py | 25 +++++++++++++++++++ 2 files changed, 62 insertions(+), 1 deletion(-) diff --git a/python/cudf/cudf/core/index.py b/python/cudf/cudf/core/index.py index 324db416be4..cd882aba297 100644 --- a/python/cudf/cudf/core/index.py +++ b/python/cudf/cudf/core/index.py @@ -1,4 +1,4 @@ -# Copyright (c) 2018-2022, NVIDIA CORPORATION. +# Copyright (c) 2018-2023, NVIDIA CORPORATION. from __future__ import annotations @@ -313,9 +313,20 @@ def copy(self, name=None, deep=False, dtype=None, names=None): Ignored for RangeIndex dtype : numpy dtype optional (default: None) Target dtype for underlying range data + + .. deprecated:: 23.02 + + The `dtype` parameter is deprecated and will be removed in + a future version of cudf. Use the `astype` method instead. + names : list-like optional (default: False) Kept compatibility with MultiIndex. Should not be used. + .. deprecated:: 23.04 + + The parameter `names` is deprecated and will be removed in + a future version of cudf. Use the `name` parameter instead. + Returns ------- New RangeIndex instance with same range, casted to new dtype @@ -327,6 +338,13 @@ def copy(self, name=None, deep=False, dtype=None, names=None): FutureWarning, ) + if names is not None: + warnings.warn( + "parameter names is deprecated and will be removed in a " + "future version. Use the name parameter instead.", + FutureWarning, + ) + dtype = self.dtype if dtype is None else dtype if not np.issubdtype(dtype, np.signedinteger): @@ -1135,9 +1153,20 @@ def copy(self, name=None, deep=False, dtype=None, names=None): With ``deep=False`` the original data is used dtype : numpy dtype, default None Target datatype to cast into, use original dtype when None + + .. deprecated:: 23.02 + + The `dtype` parameter is deprecated and will be removed in + a future version of cudf. Use the `astype` method instead. + names : list-like, default False Kept compatibility with MultiIndex. Should not be used. + .. deprecated:: 23.04 + + The parameter `names` is deprecated and will be removed in + a future version of cudf. Use the `name` parameter instead. + Returns ------- New index instance, casted to new dtype @@ -1149,6 +1178,13 @@ def copy(self, name=None, deep=False, dtype=None, names=None): FutureWarning, ) + if names is not None: + warnings.warn( + "parameter names is deprecated and will be removed in a " + "future version. Use the name parameter instead.", + FutureWarning, + ) + dtype = self.dtype if dtype is None else dtype name = self.name if name is None else name diff --git a/python/cudf/cudf/core/multiindex.py b/python/cudf/cudf/core/multiindex.py index 783c3996400..573a3f7f1d7 100644 --- a/python/cudf/cudf/core/multiindex.py +++ b/python/cudf/cudf/core/multiindex.py @@ -336,11 +336,29 @@ def copy( Names for each of the index levels. dtype : object, optional (default None) MultiIndex dtype, only supports None or object type + + .. deprecated:: 23.02 + + The `dtype` parameter is deprecated and will be removed in + a future version of cudf. Use the `astype` method instead. + levels : sequence of arrays, optional (default None) The unique labels for each level. Original values used if None. + + .. deprecated:: 23.02 + + The `levels` parameter is deprecated and will be removed in + a future version of cudf. + codes : sequence of arrays, optional (default None) Integers for each level designating which label at each location. Original values used if None. + + .. deprecated:: 23.02 + + The `codes` parameter is deprecated and will be removed in + a future version of cudf. + deep : Bool (default False) If True, `._data`, `._levels`, `._codes` will be copied. Ignored if `levels` or `codes` are specified. @@ -401,6 +419,13 @@ def copy( FutureWarning, ) + if dtype is not None: + warnings.warn( + "parameter dtype is deprecated and will be removed in a " + "future version. Use the astype method instead.", + FutureWarning, + ) + dtype = object if dtype is None else dtype if not pd.core.dtypes.common.is_object_dtype(dtype): raise TypeError("Dtype for MultiIndex only supports object type.") From a7ede21ea12586a02ca60aed705b26430134bab3 Mon Sep 17 00:00:00 2001 From: Ed Seidl Date: Mon, 6 Mar 2023 13:34:46 -0800 Subject: [PATCH 36/60] min_rows and num_rows are swapped in ComputePageSizes declaration in Parquet reader (#12886) Arguments are swapped in declaration (and thus docs as well). Args are passed as required by the definition when `ComputePageSizes` is called. Authors: - Ed Seidl (https://github.com/etseidl) Approvers: - Nghia Truong (https://github.com/ttnghia) - Yunsong Wang (https://github.com/PointKernel) URL: https://github.com/rapidsai/cudf/pull/12886 --- cpp/src/io/parquet/parquet_gpu.hpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/cpp/src/io/parquet/parquet_gpu.hpp b/cpp/src/io/parquet/parquet_gpu.hpp index c91f182c4f4..c3d3843362a 100644 --- a/cpp/src/io/parquet/parquet_gpu.hpp +++ b/cpp/src/io/parquet/parquet_gpu.hpp @@ -445,8 +445,8 @@ void BuildStringDictionaryIndex(ColumnChunkDesc* chunks, * * @param pages All pages to be decoded * @param chunks All chunks to be decoded - * @param num_rows Maximum number of rows to read * @param min_rows crop all rows below min_row + * @param num_rows Maximum number of rows to read * @param compute_num_rows If set to true, the num_rows field in PageInfo will be * computed * @param compute_string_sizes If set to true, the str_bytes field in PageInfo will @@ -455,8 +455,8 @@ void BuildStringDictionaryIndex(ColumnChunkDesc* chunks, */ void ComputePageSizes(hostdevice_vector& pages, hostdevice_vector const& chunks, - size_t num_rows, size_t min_row, + size_t num_rows, bool compute_num_rows, bool compute_string_sizes, rmm::cuda_stream_view stream); From f00baad6fb8fb436877b4904251922e05a54f87d Mon Sep 17 00:00:00 2001 From: Bradley Dice Date: Mon, 6 Mar 2023 14:57:47 -0800 Subject: [PATCH 37/60] Update to GCC 11 (#12868) This PR updates builds to use GCC 11. Authors: - Bradley Dice (https://github.com/bdice) Approvers: - Nghia Truong (https://github.com/ttnghia) - Jordan Jacobelli (https://github.com/jjacobelli) - David Wendt (https://github.com/davidwendt) URL: https://github.com/rapidsai/cudf/pull/12868 --- conda/environments/all_cuda-118_arch-x86_64.yaml | 2 +- conda/recipes/cudf/conda_build_config.yaml | 4 ++-- conda/recipes/cudf_kafka/conda_build_config.yaml | 4 ++-- conda/recipes/libcudf/conda_build_config.yaml | 4 ++-- cpp/benchmarks/sort/nested_types_common.hpp | 3 +++ dependencies.yaml | 4 ++-- 6 files changed, 12 insertions(+), 9 deletions(-) diff --git a/conda/environments/all_cuda-118_arch-x86_64.yaml b/conda/environments/all_cuda-118_arch-x86_64.yaml index 00aa1e2bc0f..72a8795fd13 100644 --- a/conda/environments/all_cuda-118_arch-x86_64.yaml +++ b/conda/environments/all_cuda-118_arch-x86_64.yaml @@ -28,7 +28,7 @@ dependencies: - doxygen=1.8.20 - fastavro>=0.22.9 - fsspec>=0.6.0 -- gcc_linux-64=9.* +- gcc_linux-64=11.* - hypothesis - ipython - libarrow=10 diff --git a/conda/recipes/cudf/conda_build_config.yaml b/conda/recipes/cudf/conda_build_config.yaml index 4feac647e8c..7494fec79a0 100644 --- a/conda/recipes/cudf/conda_build_config.yaml +++ b/conda/recipes/cudf/conda_build_config.yaml @@ -1,8 +1,8 @@ c_compiler_version: - - 9 + - 11 cxx_compiler_version: - - 9 + - 11 sysroot_version: - "2.17" diff --git a/conda/recipes/cudf_kafka/conda_build_config.yaml b/conda/recipes/cudf_kafka/conda_build_config.yaml index c049d21fd91..ccc49851a8e 100644 --- a/conda/recipes/cudf_kafka/conda_build_config.yaml +++ b/conda/recipes/cudf_kafka/conda_build_config.yaml @@ -1,8 +1,8 @@ c_compiler_version: - - 9 + - 11 cxx_compiler_version: - - 9 + - 11 sysroot_version: - "2.17" diff --git a/conda/recipes/libcudf/conda_build_config.yaml b/conda/recipes/libcudf/conda_build_config.yaml index ff8d9026aef..1111fc0a24e 100644 --- a/conda/recipes/libcudf/conda_build_config.yaml +++ b/conda/recipes/libcudf/conda_build_config.yaml @@ -1,8 +1,8 @@ c_compiler_version: - - 9 + - 11 cxx_compiler_version: - - 9 + - 11 cuda_compiler: - nvcc diff --git a/cpp/benchmarks/sort/nested_types_common.hpp b/cpp/benchmarks/sort/nested_types_common.hpp index c4851823534..fabef3a7a51 100644 --- a/cpp/benchmarks/sort/nested_types_common.hpp +++ b/cpp/benchmarks/sort/nested_types_common.hpp @@ -21,7 +21,10 @@ #include +// This error appears in GCC 11.3 and may be a compiler bug or nvbench bug. +#pragma GCC diagnostic ignored "-Wmaybe-uninitialized" #include +#pragma GCC diagnostic pop #include diff --git a/dependencies.yaml b/dependencies.yaml index 2c359d58378..accf3f00b3d 100644 --- a/dependencies.yaml +++ b/dependencies.yaml @@ -77,12 +77,12 @@ dependencies: - matrix: arch: x86_64 packages: - - &gcc_amd64 gcc_linux-64=9.* + - &gcc_amd64 gcc_linux-64=11.* - &sysroot_amd64 sysroot_linux-64==2.17 - matrix: arch: aarch64 packages: - - &gcc_aarch64 gcc_linux-aarch64=9.* + - &gcc_aarch64 gcc_linux-aarch64=11.* - &sysroot_aarch64 sysroot_linux-aarch64==2.17 - output_types: conda matrices: From 98b92a560b783e40b8865fde399d5183373a8b50 Mon Sep 17 00:00:00 2001 From: Bradley Dice Date: Mon, 6 Mar 2023 15:14:43 -0800 Subject: [PATCH 38/60] Update to protobuf>=4.21.6,<4.22. (#12864) This updates cudf's protobuf version to resolve #12830 and a downstream conflict with `ortools` (a dependency of cuopt), which requires `protobuf>=4.21.5`. Authors: - Bradley Dice (https://github.com/bdice) Approvers: - GALI PREM SAGAR (https://github.com/galipremsagar) - Jordan Jacobelli (https://github.com/jjacobelli) URL: https://github.com/rapidsai/cudf/pull/12864 --- conda/environments/all_cuda-118_arch-x86_64.yaml | 2 +- conda/recipes/cudf/meta.yaml | 4 ++-- dependencies.yaml | 2 +- python/cudf/pyproject.toml | 2 +- 4 files changed, 5 insertions(+), 5 deletions(-) diff --git a/conda/environments/all_cuda-118_arch-x86_64.yaml b/conda/environments/all_cuda-118_arch-x86_64.yaml index 72a8795fd13..67e2dc4720e 100644 --- a/conda/environments/all_cuda-118_arch-x86_64.yaml +++ b/conda/environments/all_cuda-118_arch-x86_64.yaml @@ -50,7 +50,7 @@ dependencies: - pandoc<=2.0.0 - pip - pre-commit -- protobuf=4.21 +- protobuf>=4.21.6,<4.22 - ptxcompiler - pyarrow=10 - pydata-sphinx-theme diff --git a/conda/recipes/cudf/meta.yaml b/conda/recipes/cudf/meta.yaml index 27073eb323b..1ce401abb81 100644 --- a/conda/recipes/cudf/meta.yaml +++ b/conda/recipes/cudf/meta.yaml @@ -45,7 +45,7 @@ requirements: - ninja - sysroot_{{ target_platform }} {{ sysroot_version }} host: - - protobuf =4.21 + - protobuf >=4.21.6,<4.22 - python - cython >=0.29,<0.30 - scikit-build >=0.13.1 @@ -57,7 +57,7 @@ requirements: - rmm ={{ minor_version }} - cudatoolkit ={{ cuda_version }} run: - - protobuf =4.21 + - protobuf >=4.21.6,<4.22 - python - typing_extensions - pandas >=1.0,<1.6.0dev0 diff --git a/dependencies.yaml b/dependencies.yaml index accf3f00b3d..4bac8148b10 100644 --- a/dependencies.yaml +++ b/dependencies.yaml @@ -115,7 +115,7 @@ dependencies: - scikit-build>=0.13.1 - output_types: conda packages: - - protobuf=4.21 + - protobuf>=4.21.6,<4.22 cudatoolkit: specific: - output_types: conda diff --git a/python/cudf/pyproject.toml b/python/cudf/pyproject.toml index 0dc719a0846..ca14ccfc63e 100644 --- a/python/cudf/pyproject.toml +++ b/python/cudf/pyproject.toml @@ -35,7 +35,7 @@ dependencies = [ "nvtx>=0.2.1", "packaging", "pandas>=1.0,<1.6.0dev0", - "protobuf==4.21", + "protobuf>=4.21.6,<4.22", "typing_extensions", # Allow floating minor versions for Arrow. "pyarrow==10", From 0ee82c9619ef2145ea270c6c97eb5c5060eb2884 Mon Sep 17 00:00:00 2001 From: Peixin Date: Tue, 7 Mar 2023 13:09:44 +0800 Subject: [PATCH 39/60] Update JNI build ENV default to gcc 11 (#12881) related to #2568 update JNI dockerfile default to gcc 11. mark as draft for review first, will verify after https://github.com/rapidsai/cudf/pull/12868 Authors: - Peixin (https://github.com/pxLi) Approvers: - Jason Lowe (https://github.com/jlowe) URL: https://github.com/rapidsai/cudf/pull/12881 --- java/ci/Dockerfile.centos7 | 4 ++-- java/ci/README.md | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/java/ci/Dockerfile.centos7 b/java/ci/Dockerfile.centos7 index dd6b161ac24..65f151a5126 100644 --- a/java/ci/Dockerfile.centos7 +++ b/java/ci/Dockerfile.centos7 @@ -1,5 +1,5 @@ # -# Copyright (c) 2020-2022, NVIDIA CORPORATION. All rights reserved. +# Copyright (c) 2020-2023, NVIDIA CORPORATION. All rights reserved. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. @@ -24,7 +24,7 @@ ARG CUDA_VERSION=11.8.0 FROM nvidia/cuda:$CUDA_VERSION-devel-centos7 ### Install basic requirements -ARG DEVTOOLSET_VERSION=9 +ARG DEVTOOLSET_VERSION=11 RUN yum install -y centos-release-scl RUN yum install -y devtoolset-${DEVTOOLSET_VERSION} epel-release RUN yum install -y git zlib-devel maven tar wget patch ninja-build diff --git a/java/ci/README.md b/java/ci/README.md index ee2c7c24555..daf0bedd6c5 100644 --- a/java/ci/README.md +++ b/java/ci/README.md @@ -42,7 +42,7 @@ git clone --recursive https://github.com/rapidsai/cudf.git -b branch-23.04 ```bash cd cudf export WORKSPACE=`pwd` -scl enable devtoolset-9 "java/ci/build-in-docker.sh" +scl enable devtoolset-11 "java/ci/build-in-docker.sh" ``` ### The output From a78fdd51ca994d2072085f5c70b33fc3452a3484 Mon Sep 17 00:00:00 2001 From: David Wendt <45795991+davidwendt@users.noreply.github.com> Date: Tue, 7 Mar 2023 07:54:41 -0500 Subject: [PATCH 40/60] Fix cudf::hash_partition kernel launch error with decimal128 types (#12863) Fixes `cudf::hash_partition` error when using `decimal128` column types. The internal optimized path, `copy_block_partitions`, uses shared-memory for copying fixed-width type column elements. For `int128_t` type, the shared-memory needed (~64KB) is larger than the maximum size (~48KB) allowed causing a kernel launch failure. The optimized path is now restricted to only fixed-width types `int64_t` and below. The `int128_t` column types will fall through to the gather-map pattern instead. Accommodating this type in the existing copy-block implementation would likely penalize the performance of the other fixed-width types. If the new implementation becomes insufficient, we could explore a special optimized path in the future for the single type `int128_t`. An existing gtest for fixed-point types was updated to include a `decimal128` column to catch this kind of error in the future. Closes #12852 Authors: - David Wendt (https://github.com/davidwendt) - GALI PREM SAGAR (https://github.com/galipremsagar) Approvers: - Nghia Truong (https://github.com/ttnghia) - Divye Gala (https://github.com/divyegala) - Bradley Dice (https://github.com/bdice) URL: https://github.com/rapidsai/cudf/pull/12863 --- cpp/src/partitioning/partitioning.cu | 24 +++++++++++++------ .../partitioning/hash_partition_test.cpp | 14 +++++------ 2 files changed, 24 insertions(+), 14 deletions(-) diff --git a/cpp/src/partitioning/partitioning.cu b/cpp/src/partitioning/partitioning.cu index edf5d6d6612..54dffc85aca 100644 --- a/cpp/src/partitioning/partitioning.cu +++ b/cpp/src/partitioning/partitioning.cu @@ -14,7 +14,6 @@ * limitations under the License. */ -#include #include #include #include @@ -36,6 +35,9 @@ #include #include +#include +#include + namespace cudf { namespace { // Launch configuration for optimized hash partition @@ -389,7 +391,15 @@ rmm::device_uvector compute_gather_map(size_type num_rows, } struct copy_block_partitions_dispatcher { - template ()>* = nullptr> + template + constexpr static bool is_copy_block_supported() + { + // The shared-memory used for fixed-width types in the copy_block_partitions_impl function + // will be too large for any DataType greater than int64_t. + return is_fixed_width() && (sizeof(DataType) <= sizeof(int64_t)); + } + + template ())> std::unique_ptr operator()(column_view const& input, const size_type num_partitions, size_type const* row_partition_numbers, @@ -416,7 +426,7 @@ struct copy_block_partitions_dispatcher { return std::make_unique(input.type(), input.size(), std::move(output)); } - template ()>* = nullptr> + template ())> std::unique_ptr operator()(column_view const& input, const size_type num_partitions, size_type const* row_partition_numbers, @@ -713,7 +723,7 @@ struct dispatch_map_type { } // namespace namespace detail { -namespace local { +namespace { template