Skip to content

Commit

Permalink
Add version to deprecation messages (#6782)
Browse files Browse the repository at this point in the history
Version is inferred from first release tag containing the commit that
added the deprecation.
  • Loading branch information
findepi authored Nov 24, 2024
1 parent 9705563 commit f033e4f
Show file tree
Hide file tree
Showing 29 changed files with 49 additions and 38 deletions.
4 changes: 2 additions & 2 deletions arrow-arith/src/arity.rs
Original file line number Diff line number Diff line change
Expand Up @@ -104,7 +104,7 @@ where
}

/// Applies an infallible unary function to an array with primitive values.
#[deprecated(note = "Use arrow_array::AnyDictionaryArray")]
#[deprecated(since = "46.0.0", note = "Use arrow_array::AnyDictionaryArray")]
pub fn unary_dyn<F, T>(array: &dyn Array, op: F) -> Result<ArrayRef, ArrowError>
where
T: ArrowPrimitiveType,
Expand All @@ -130,7 +130,7 @@ where
}

/// Applies a fallible unary function to an array with primitive values.
#[deprecated(note = "Use arrow_array::AnyDictionaryArray")]
#[deprecated(since = "46.0.0", note = "Use arrow_array::AnyDictionaryArray")]
pub fn try_unary_dyn<F, T>(array: &dyn Array, op: F) -> Result<ArrayRef, ArrowError>
where
T: ArrowPrimitiveType,
Expand Down
2 changes: 1 addition & 1 deletion arrow-arith/src/temporal.rs
Original file line number Diff line number Diff line change
Expand Up @@ -669,7 +669,7 @@ impl<T: Datelike> ChronoDateExt for T {
///
/// Note that the offset is function of time and can vary depending on whether daylight savings is
/// in effect or not. e.g. Australia/Sydney is +10:00 or +11:00 depending on DST.
#[deprecated(note = "Use arrow_array::timezone::Tz instead")]
#[deprecated(since = "26.0.0", note = "Use arrow_array::timezone::Tz instead")]
pub fn using_chrono_tz_and_utc_naive_date_time(
tz: &str,
utc: NaiveDateTime,
Expand Down
2 changes: 1 addition & 1 deletion arrow-array/src/array/binary_array.rs
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,7 @@ pub type GenericBinaryArray<OffsetSize> = GenericByteArray<GenericBinaryType<Off

impl<OffsetSize: OffsetSizeTrait> GenericBinaryArray<OffsetSize> {
/// Get the data type of the array.
#[deprecated(note = "please use `Self::DATA_TYPE` instead")]
#[deprecated(since = "20.0.0", note = "please use `Self::DATA_TYPE` instead")]
pub const fn get_data_type() -> DataType {
Self::DATA_TYPE
}
Expand Down
1 change: 1 addition & 0 deletions arrow-array/src/array/fixed_size_binary_array.rs
Original file line number Diff line number Diff line change
Expand Up @@ -237,6 +237,7 @@ impl FixedSizeBinaryArray {
///
/// Returns error if argument has length zero, or sizes of nested slices don't match.
#[deprecated(
since = "28.0.0",
note = "This function will fail if the iterator produces only None values; prefer `try_from_sparse_iter_with_size`"
)]
pub fn try_from_sparse_iter<T, U>(mut iter: T) -> Result<Self, ArrowError>
Expand Down
4 changes: 2 additions & 2 deletions arrow-array/src/array/primitive_array.rs
Original file line number Diff line number Diff line change
Expand Up @@ -1481,7 +1481,7 @@ def_numeric_from_vec!(TimestampNanosecondType);

impl<T: ArrowTimestampType> PrimitiveArray<T> {
/// Construct a timestamp array from a vec of i64 values and an optional timezone
#[deprecated(note = "Use with_timezone_opt instead")]
#[deprecated(since = "26.0.0", note = "Use with_timezone_opt instead")]
pub fn from_vec(data: Vec<i64>, timezone: Option<String>) -> Self
where
Self: From<Vec<i64>>,
Expand All @@ -1490,7 +1490,7 @@ impl<T: ArrowTimestampType> PrimitiveArray<T> {
}

/// Construct a timestamp array from a vec of `Option<i64>` values and an optional timezone
#[deprecated(note = "Use with_timezone_opt instead")]
#[deprecated(since = "26.0.0", note = "Use with_timezone_opt instead")]
pub fn from_opt_vec(data: Vec<Option<i64>>, timezone: Option<String>) -> Self
where
Self: From<Vec<Option<i64>>>,
Expand Down
2 changes: 1 addition & 1 deletion arrow-array/src/array/string_array.rs
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,7 @@ pub type GenericStringArray<OffsetSize> = GenericByteArray<GenericStringType<Off

impl<OffsetSize: OffsetSizeTrait> GenericStringArray<OffsetSize> {
/// Get the data type of the array.
#[deprecated(note = "please use `Self::DATA_TYPE` instead")]
#[deprecated(since = "20.0.0", note = "please use `Self::DATA_TYPE` instead")]
pub const fn get_data_type() -> DataType {
Self::DATA_TYPE
}
Expand Down
2 changes: 1 addition & 1 deletion arrow-array/src/builder/generic_bytes_view_builder.rs
Original file line number Diff line number Diff line change
Expand Up @@ -136,7 +136,7 @@ impl<T: ByteViewType + ?Sized> GenericByteViewBuilder<T> {

/// Override the size of buffers to allocate for holding string data
/// Use `with_fixed_block_size` instead.
#[deprecated(note = "Use `with_fixed_block_size` instead")]
#[deprecated(since = "53.0.0", note = "Use `with_fixed_block_size` instead")]
pub fn with_block_size(self, block_size: u32) -> Self {
self.with_fixed_block_size(block_size)
}
Expand Down
5 changes: 4 additions & 1 deletion arrow-array/src/cast.rs
Original file line number Diff line number Diff line change
Expand Up @@ -690,7 +690,10 @@ array_downcast_fn!(as_union_array, UnionArray);
array_downcast_fn!(as_map_array, MapArray);

/// Force downcast of an Array, such as an ArrayRef to Decimal128Array, panic’ing on failure.
#[deprecated(note = "please use `as_primitive_array::<Decimal128Type>` instead")]
#[deprecated(
since = "42.0.0",
note = "please use `as_primitive_array::<Decimal128Type>` instead"
)]
pub fn as_decimal_array(arr: &dyn Array) -> &PrimitiveArray<Decimal128Type> {
as_primitive_array::<Decimal128Type>(arr)
}
Expand Down
5 changes: 4 additions & 1 deletion arrow-array/src/ffi.rs
Original file line number Diff line number Diff line change
Expand Up @@ -121,7 +121,10 @@ type Result<T> = std::result::Result<T, ArrowError>;
/// This function copies the content of two FFI structs [arrow_data::ffi::FFI_ArrowArray] and
/// [arrow_schema::ffi::FFI_ArrowSchema] in the array to the location pointed by the raw pointers.
/// Usually the raw pointers are provided by the array data consumer.
#[deprecated(note = "Use FFI_ArrowArray::new and FFI_ArrowSchema::try_from")]
#[deprecated(
since = "52.0.0",
note = "Use FFI_ArrowArray::new and FFI_ArrowSchema::try_from"
)]
pub unsafe fn export_array_into_raw(
src: ArrayRef,
out_array: *mut FFI_ArrowArray,
Expand Down
2 changes: 1 addition & 1 deletion arrow-array/src/ffi_stream.rs
Original file line number Diff line number Diff line change
Expand Up @@ -384,7 +384,7 @@ impl RecordBatchReader for ArrowArrayStreamReader {
/// # Safety
/// Assumes that the pointer represents valid C Stream Interfaces, both in memory
/// representation and lifetime via the `release` mechanism.
#[deprecated(note = "Use FFI_ArrowArrayStream::new")]
#[deprecated(since = "50.0.0", note = "Use FFI_ArrowArrayStream::new")]
pub unsafe fn export_reader_into_raw(
reader: Box<dyn RecordBatchReader + Send>,
out_stream: *mut FFI_ArrowArrayStream,
Expand Down
4 changes: 2 additions & 2 deletions arrow-array/src/types.rs
Original file line number Diff line number Diff line change
Expand Up @@ -69,7 +69,7 @@ pub trait ArrowPrimitiveType: primitive::PrimitiveTypeSealed + 'static {
const DATA_TYPE: DataType;

/// Returns the byte width of this primitive type.
#[deprecated(note = "Use ArrowNativeType::get_byte_width")]
#[deprecated(since = "52.0.0", note = "Use ArrowNativeType::get_byte_width")]
fn get_byte_width() -> usize {
std::mem::size_of::<Self::Native>()
}
Expand Down Expand Up @@ -325,7 +325,7 @@ pub trait ArrowTimestampType: ArrowTemporalType<Native = i64> {
const UNIT: TimeUnit;

/// Returns the `TimeUnit` of this timestamp.
#[deprecated(note = "Use Self::UNIT")]
#[deprecated(since = "36.0.0", note = "Use Self::UNIT")]
fn get_time_unit() -> TimeUnit {
Self::UNIT
}
Expand Down
2 changes: 1 addition & 1 deletion arrow-buffer/src/buffer/boolean.rs
Original file line number Diff line number Diff line change
Expand Up @@ -102,7 +102,7 @@ impl BooleanBuffer {
///
/// Panics if `i >= self.len()`
#[inline]
#[deprecated(note = "use BooleanBuffer::value")]
#[deprecated(since = "36.0.0", note = "use BooleanBuffer::value")]
pub fn is_set(&self, i: usize) -> bool {
self.value(i)
}
Expand Down
4 changes: 2 additions & 2 deletions arrow-buffer/src/buffer/immutable.rs
Original file line number Diff line number Diff line change
Expand Up @@ -112,7 +112,7 @@ impl Buffer {
///
/// This function is unsafe as there is no guarantee that the given pointer is valid for `len`
/// bytes. If the `ptr` and `capacity` come from a `Buffer`, then this is guaranteed.
#[deprecated(note = "Use Buffer::from_vec")]
#[deprecated(since = "50.0.0", note = "Use Buffer::from_vec")]
pub unsafe fn from_raw_parts(ptr: NonNull<u8>, len: usize, capacity: usize) -> Self {
assert!(len <= capacity);
let layout = Layout::from_size_align(capacity, ALIGNMENT).unwrap();
Expand Down Expand Up @@ -279,7 +279,7 @@ impl Buffer {
}

/// Returns the number of 1-bits in this buffer.
#[deprecated(note = "use count_set_bits_offset instead")]
#[deprecated(since = "27.0.0", note = "use count_set_bits_offset instead")]
pub fn count_set_bits(&self) -> usize {
let len_in_bits = self.len() * 8;
// self.offset is already taken into consideration by the bit_chunks implementation
Expand Down
2 changes: 1 addition & 1 deletion arrow-buffer/src/buffer/mutable.rs
Original file line number Diff line number Diff line change
Expand Up @@ -120,7 +120,7 @@ impl MutableBuffer {

/// Create a [`MutableBuffer`] from the provided [`Vec`] without copying
#[inline]
#[deprecated(note = "Use From<Vec<T>>")]
#[deprecated(since = "46.0.0", note = "Use From<Vec<T>>")]
pub fn from_vec<T: ArrowNativeType>(vec: Vec<T>) -> Self {
Self::from(vec)
}
Expand Down
6 changes: 3 additions & 3 deletions arrow-buffer/src/native.rs
Original file line number Diff line number Diff line change
Expand Up @@ -92,23 +92,23 @@ pub trait ArrowNativeType:
/// Convert native type from i32.
///
/// Returns `None` if [`Self`] is not `i32`
#[deprecated(note = "please use `Option::Some` instead")]
#[deprecated(since = "24.0.0", note = "please use `Option::Some` instead")]
fn from_i32(_: i32) -> Option<Self> {
None
}

/// Convert native type from i64.
///
/// Returns `None` if [`Self`] is not `i64`
#[deprecated(note = "please use `Option::Some` instead")]
#[deprecated(since = "24.0.0", note = "please use `Option::Some` instead")]
fn from_i64(_: i64) -> Option<Self> {
None
}

/// Convert native type from i128.
///
/// Returns `None` if [`Self`] is not `i128`
#[deprecated(note = "please use `Option::Some` instead")]
#[deprecated(since = "24.0.0", note = "please use `Option::Some` instead")]
fn from_i128(_: i128) -> Option<Self> {
None
}
Expand Down
6 changes: 3 additions & 3 deletions arrow-csv/src/reader/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -408,7 +408,7 @@ impl Format {
/// reader cursor offset.
///
/// The inferred schema will always have each field set as nullable.
#[deprecated(note = "Use Format::infer_schema")]
#[deprecated(since = "39.0.0", note = "Use Format::infer_schema")]
#[allow(deprecated)]
pub fn infer_file_schema<R: Read + Seek>(
mut reader: R,
Expand All @@ -429,7 +429,7 @@ pub fn infer_file_schema<R: Read + Seek>(
/// not set, all records are read to infer the schema.
///
/// Return inferred schema and number of records used for inference.
#[deprecated(note = "Use Format::infer_schema")]
#[deprecated(since = "39.0.0", note = "Use Format::infer_schema")]
pub fn infer_reader_schema<R: Read>(
reader: R,
delimiter: u8,
Expand Down Expand Up @@ -1102,7 +1102,7 @@ impl ReaderBuilder {
}

/// Set whether the CSV file has headers
#[deprecated(note = "Use with_header")]
#[deprecated(since = "39.0.0", note = "Use with_header")]
#[doc(hidden)]
pub fn has_header(mut self, has_header: bool) -> Self {
self.format.header = has_header;
Expand Down
4 changes: 2 additions & 2 deletions arrow-csv/src/writer.rs
Original file line number Diff line number Diff line change
Expand Up @@ -257,7 +257,7 @@ impl WriterBuilder {
}

/// Set whether to write headers
#[deprecated(note = "Use Self::with_header")]
#[deprecated(since = "39.0.0", note = "Use Self::with_header")]
#[doc(hidden)]
pub fn has_headers(mut self, has_headers: bool) -> Self {
self.has_header = has_headers;
Expand Down Expand Up @@ -398,7 +398,7 @@ impl WriterBuilder {
}

/// Use RFC3339 format for date/time/timestamps (default)
#[deprecated(note = "Use WriterBuilder::default()")]
#[deprecated(since = "39.0.0", note = "Use WriterBuilder::default()")]
pub fn with_rfc3339(mut self) -> Self {
self.date_format = None;
self.datetime_format = None;
Expand Down
2 changes: 1 addition & 1 deletion arrow-data/src/data.rs
Original file line number Diff line number Diff line change
Expand Up @@ -32,7 +32,7 @@ use crate::{equal, validate_binary_view, validate_string_view};

/// A collection of [`Buffer`]
#[doc(hidden)]
#[deprecated(note = "Use [Buffer]")]
#[deprecated(since = "46.0.0", note = "Use [Buffer]")]
pub type Buffers<'a> = &'a [Buffer];

#[inline]
Expand Down
2 changes: 1 addition & 1 deletion arrow-json/src/reader/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -246,7 +246,7 @@ impl ReaderBuilder {

/// Sets if the decoder should coerce primitive values (bool and number) into string
/// when the Schema's column is Utf8 or LargeUtf8.
#[deprecated(note = "Use with_coerce_primitive")]
#[deprecated(since = "39.0.0", note = "Use with_coerce_primitive")]
pub fn coerce_primitive(self, coerce_primitive: bool) -> Self {
self.with_coerce_primitive(coerce_primitive)
}
Expand Down
5 changes: 4 additions & 1 deletion arrow-ord/src/cmp.rs
Original file line number Diff line number Diff line change
Expand Up @@ -656,7 +656,10 @@ pub fn compare_byte_view<T: ByteViewType>(
///
/// # Safety
/// The left/right_idx must within range of each array
#[deprecated(note = "Use `GenericByteViewArray::compare_unchecked` instead")]
#[deprecated(
since = "52.2.0",
note = "Use `GenericByteViewArray::compare_unchecked` instead"
)]
pub unsafe fn compare_byte_view_unchecked<T: ByteViewType>(
left: &GenericByteViewArray<T>,
left_idx: usize,
Expand Down
2 changes: 1 addition & 1 deletion arrow-ord/src/ord.rs
Original file line number Diff line number Diff line change
Expand Up @@ -265,7 +265,7 @@ fn compare_struct(
Ok(f)
}

#[deprecated(note = "Use make_comparator")]
#[deprecated(since = "52.0.0", note = "Use make_comparator")]
#[doc(hidden)]
pub fn build_compare(left: &dyn Array, right: &dyn Array) -> Result<DynComparator, ArrowError> {
make_comparator(left, right, SortOptions::default())
Expand Down
2 changes: 1 addition & 1 deletion arrow-ord/src/partition.rs
Original file line number Diff line number Diff line change
Expand Up @@ -167,7 +167,7 @@ fn find_boundaries(v: &dyn Array) -> Result<BooleanBuffer, ArrowError> {
/// The returned vec would be of size k where k is cardinality of the sorted values; Consecutive
/// values will be connected: (a, b) and (b, c), where start = 0 and end = n for the first and last
/// range.
#[deprecated(note = "Use partition")]
#[deprecated(since = "46.0.0", note = "Use partition")]
pub fn lexicographical_partition_ranges(
columns: &[SortColumn],
) -> Result<impl Iterator<Item = Range<usize>> + '_, ArrowError> {
Expand Down
2 changes: 1 addition & 1 deletion arrow-schema/src/fields.rs
Original file line number Diff line number Diff line change
Expand Up @@ -275,7 +275,7 @@ impl Fields {
/// assert_eq!(fields.remove(1), Field::new("b", DataType::Int8, false).into());
/// assert_eq!(fields.len(), 2);
/// ```
#[deprecated(note = "Use SchemaBuilder::remove")]
#[deprecated(since = "50.0.0", note = "Use SchemaBuilder::remove")]
#[doc(hidden)]
pub fn remove(&mut self, index: usize) -> FieldRef {
let mut builder = SchemaBuilder::from(Fields::from(&*self.0));
Expand Down
2 changes: 1 addition & 1 deletion arrow-schema/src/schema.rs
Original file line number Diff line number Diff line change
Expand Up @@ -455,7 +455,7 @@ impl Schema {
/// assert_eq!(schema.remove(1), Field::new("b", DataType::Int8, false).into());
/// assert_eq!(schema.fields.len(), 2);
/// ```
#[deprecated(note = "Use SchemaBuilder::remove")]
#[deprecated(since = "50.0.0", note = "Use SchemaBuilder::remove")]
#[doc(hidden)]
#[allow(deprecated)]
pub fn remove(&mut self, index: usize) -> FieldRef {
Expand Down
2 changes: 1 addition & 1 deletion object_store/src/memory.rs
Original file line number Diff line number Diff line change
Expand Up @@ -469,7 +469,7 @@ impl InMemory {
}

/// Creates a clone of the store
#[deprecated(note = "Use fork() instead")]
#[deprecated(since = "44.0.0", note = "Use fork() instead")]
pub async fn clone(&self) -> Self {
self.fork()
}
Expand Down
2 changes: 1 addition & 1 deletion object_store/src/prefix.rs
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,7 @@ use crate::{
};

#[doc(hidden)]
#[deprecated(note = "Use PrefixStore")]
#[deprecated(since = "36.0.0", note = "Use PrefixStore")]
pub type PrefixObjectStore<T> = PrefixStore<T>;

/// Store wrapper that applies a constant prefix to all paths handled by the store.
Expand Down
1 change: 1 addition & 0 deletions parquet/src/basic.rs
Original file line number Diff line number Diff line change
Expand Up @@ -302,6 +302,7 @@ pub enum Encoding {
///
/// The RLE/bit-packing hybrid is more cpu and memory efficient and should be used instead.
#[deprecated(
since = "51.0.0",
note = "Please see documentation for compatibility issues and use the RLE/bit-packing hybrid encoding instead"
)]
BIT_PACKED,
Expand Down
2 changes: 1 addition & 1 deletion parquet/src/column/reader.rs
Original file line number Diff line number Diff line change
Expand Up @@ -197,7 +197,7 @@ where
///
/// `values` will be contiguously populated with the non-null values. Note that if the column
/// is not required, this may be less than either `batch_size` or the number of levels read
#[deprecated(note = "Use read_records")]
#[deprecated(since = "42.0.0", note = "Use read_records")]
pub fn read_batch(
&mut self,
batch_size: usize,
Expand Down
6 changes: 3 additions & 3 deletions parquet/src/file/metadata/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -190,7 +190,7 @@ impl ParquetMetaData {

/// Creates Parquet metadata from file metadata, a list of row
/// group metadata, and the column index structures.
#[deprecated(note = "Use ParquetMetaDataBuilder")]
#[deprecated(since = "53.1.0", note = "Use ParquetMetaDataBuilder")]
pub fn new_with_page_index(
file_metadata: FileMetaData,
row_groups: Vec<RowGroupMetaData>,
Expand Down Expand Up @@ -231,7 +231,7 @@ impl ParquetMetaData {
}

/// Returns page indexes in this file.
#[deprecated(note = "Use Self::column_index")]
#[deprecated(since = "39.0.0", note = "Use Self::column_index")]
pub fn page_indexes(&self) -> Option<&ParquetColumnIndex> {
self.column_index.as_ref()
}
Expand All @@ -247,7 +247,7 @@ impl ParquetMetaData {
}

/// Returns the offset index for this file if loaded
#[deprecated(note = "Use Self::offset_index")]
#[deprecated(since = "39.0.0", note = "Use Self::offset_index")]
pub fn offset_indexes(&self) -> Option<&ParquetOffsetIndex> {
self.offset_index.as_ref()
}
Expand Down

0 comments on commit f033e4f

Please sign in to comment.