You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by tu...@apache.org on 2022/08/18 16:42:48 UTC

[arrow-rs] branch master updated: RFC: Simplify decimal (#2440) (#2477)

This is an automated email from the ASF dual-hosted git repository.

tustvold pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/arrow-rs.git


The following commit(s) were added to refs/heads/master by this push:
     new 15f42b26f RFC: Simplify decimal (#2440) (#2477)
15f42b26f is described below

commit 15f42b26f80833b5fe9cb7fe72b37492950eadb9
Author: Raphael Taylor-Davies <17...@users.noreply.github.com>
AuthorDate: Thu Aug 18 17:42:44 2022 +0100

    RFC: Simplify decimal (#2440) (#2477)
    
    * Simplify decimal (#2440)
    
    * Format
    
    * Fix doc
    
    * Review feedback
    
    * Add docs
    
    * Fix logical merge conflict
---
 arrow/src/array/array_decimal.rs           | 112 ++++++++++++++--------------
 arrow/src/array/builder/decimal_builder.rs |   2 +-
 arrow/src/array/iterator.rs                |  70 ++----------------
 arrow/src/array/mod.rs                     |   5 +-
 arrow/src/datatypes/types.rs               |  72 ++++++++++++++++++
 arrow/src/util/decimal.rs                  | 114 +++++++++++++++--------------
 6 files changed, 195 insertions(+), 180 deletions(-)

diff --git a/arrow/src/array/array_decimal.rs b/arrow/src/array/array_decimal.rs
index 4af3d1dba..540024a92 100644
--- a/arrow/src/array/array_decimal.rs
+++ b/arrow/src/array/array_decimal.rs
@@ -18,19 +18,22 @@
 use crate::array::ArrayAccessor;
 use std::convert::From;
 use std::fmt;
+use std::marker::PhantomData;
 use std::{any::Any, iter::FromIterator};
 
 use super::{
     array::print_long_array, raw_pointer::RawPtrBox, Array, ArrayData, FixedSizeListArray,
 };
-use super::{BasicDecimalIter, BooleanBufferBuilder, FixedSizeBinaryArray};
+use super::{BooleanBufferBuilder, DecimalIter, FixedSizeBinaryArray};
 #[allow(deprecated)]
-pub use crate::array::DecimalIter;
 use crate::buffer::{Buffer, MutableBuffer};
 use crate::datatypes::validate_decimal_precision;
-use crate::datatypes::{validate_decimal256_precision_with_lt_bytes, DataType};
+use crate::datatypes::{
+    validate_decimal256_precision_with_lt_bytes, DataType, Decimal128Type,
+    Decimal256Type, DecimalType, NativeDecimalType,
+};
 use crate::error::{ArrowError, Result};
-use crate::util::decimal::{BasicDecimal, Decimal256};
+use crate::util::decimal::{Decimal, Decimal256};
 
 /// `Decimal128Array` stores fixed width decimal numbers,
 /// with a fixed precision and scale.
@@ -38,7 +41,7 @@ use crate::util::decimal::{BasicDecimal, Decimal256};
 /// # Examples
 ///
 /// ```
-///    use arrow::array::{Array, BasicDecimalArray, Decimal128Array};
+///    use arrow::array::{Array, DecimalArray, Decimal128Array};
 ///    use arrow::datatypes::DataType;
 ///
 ///    // Create a DecimalArray with the default precision and scale
@@ -66,24 +69,29 @@ use crate::util::decimal::{BasicDecimal, Decimal256};
 ///    assert_eq!(6, decimal_array.scale());
 /// ```
 ///
-pub type Decimal128Array = BasicDecimalArray<16>;
+pub type Decimal128Array = DecimalArray<Decimal128Type>;
 
-pub type Decimal256Array = BasicDecimalArray<32>;
+/// `Decimal256Array` stores fixed width decimal numbers,
+/// with a fixed precision and scale
+pub type Decimal256Array = DecimalArray<Decimal256Type>;
 
-pub struct BasicDecimalArray<const BYTE_WIDTH: usize> {
+/// A generic [`Array`] for fixed width decimal numbers
+///
+/// See [`Decimal128Array`] and [`Decimal256Array`]
+pub struct DecimalArray<T: DecimalType> {
     data: ArrayData,
     value_data: RawPtrBox<u8>,
     precision: usize,
     scale: usize,
+    _phantom: PhantomData<T>,
 }
 
-impl<const BYTE_WIDTH: usize> BasicDecimalArray<BYTE_WIDTH> {
-    pub const VALUE_LENGTH: i32 = BYTE_WIDTH as i32;
-    const DEFAULT_TYPE: DataType = BasicDecimal::<BYTE_WIDTH>::DEFAULT_TYPE;
-    pub const MAX_PRECISION: usize = BasicDecimal::<BYTE_WIDTH>::MAX_PRECISION;
-    pub const MAX_SCALE: usize = BasicDecimal::<BYTE_WIDTH>::MAX_SCALE;
-    const TYPE_CONSTRUCTOR: fn(usize, usize) -> DataType =
-        BasicDecimal::<BYTE_WIDTH>::TYPE_CONSTRUCTOR;
+impl<T: DecimalType> DecimalArray<T> {
+    pub const VALUE_LENGTH: i32 = T::BYTE_LENGTH as i32;
+    const DEFAULT_TYPE: DataType = T::DEFAULT_TYPE;
+    pub const MAX_PRECISION: usize = T::MAX_PRECISION;
+    pub const MAX_SCALE: usize = T::MAX_SCALE;
+    const TYPE_CONSTRUCTOR: fn(usize, usize) -> DataType = T::TYPE_CONSTRUCTOR;
 
     pub fn data(&self) -> &ArrayData {
         &self.data
@@ -100,7 +108,7 @@ impl<const BYTE_WIDTH: usize> BasicDecimalArray<BYTE_WIDTH> {
     }
 
     /// Returns the element at index `i`.
-    pub fn value(&self, i: usize) -> BasicDecimal<BYTE_WIDTH> {
+    pub fn value(&self, i: usize) -> Decimal<T> {
         assert!(i < self.data().len(), "Out of bounds access");
 
         unsafe { self.value_unchecked(i) }
@@ -109,19 +117,17 @@ impl<const BYTE_WIDTH: usize> BasicDecimalArray<BYTE_WIDTH> {
     /// Returns the element at index `i`.
     /// # Safety
     /// Caller is responsible for ensuring that the index is within the bounds of the array
-    pub unsafe fn value_unchecked(&self, i: usize) -> BasicDecimal<BYTE_WIDTH> {
+    pub unsafe fn value_unchecked(&self, i: usize) -> Decimal<T> {
         let data = self.data();
         let offset = i + data.offset();
         let raw_val = {
             let pos = self.value_offset_at(offset);
-            std::slice::from_raw_parts(
+            T::Native::from_slice(std::slice::from_raw_parts(
                 self.raw_value_data_ptr().offset(pos as isize),
                 Self::VALUE_LENGTH as usize,
-            )
-            .try_into()
-            .unwrap()
+            ))
         };
-        BasicDecimal::<BYTE_WIDTH>::new(self.precision(), self.scale(), raw_val)
+        Decimal::new(self.precision(), self.scale(), &raw_val)
     }
 
     /// Returns the offset for the element at index `i`.
@@ -304,7 +310,8 @@ impl<const BYTE_WIDTH: usize> BasicDecimalArray<BYTE_WIDTH> {
 
     // validate all the data in the array are valid within the new precision or not
     fn validate_data(&self, precision: usize) -> Result<()> {
-        match BYTE_WIDTH {
+        // TODO: Move into DecimalType
+        match Self::VALUE_LENGTH {
             16 => self
                 .as_any()
                 .downcast_ref::<Decimal128Array>()
@@ -376,7 +383,7 @@ impl Decimal256Array {
     }
 }
 
-impl<const BYTE_WIDTH: usize> From<ArrayData> for BasicDecimalArray<BYTE_WIDTH> {
+impl<T: DecimalType> From<ArrayData> for DecimalArray<T> {
     fn from(data: ArrayData) -> Self {
         assert_eq!(
             data.buffers().len(),
@@ -384,7 +391,7 @@ impl<const BYTE_WIDTH: usize> From<ArrayData> for BasicDecimalArray<BYTE_WIDTH>
             "DecimalArray data should contain 1 buffer only (values)"
         );
         let values = data.buffers()[0].as_ptr();
-        let (precision, scale) = match (data.data_type(), BYTE_WIDTH) {
+        let (precision, scale) = match (data.data_type(), Self::VALUE_LENGTH) {
             (DataType::Decimal128(precision, scale), 16)
             | (DataType::Decimal256(precision, scale), 32) => (*precision, *scale),
             _ => panic!("Expected data type to be Decimal"),
@@ -394,27 +401,18 @@ impl<const BYTE_WIDTH: usize> From<ArrayData> for BasicDecimalArray<BYTE_WIDTH>
             value_data: unsafe { RawPtrBox::new(values) },
             precision,
             scale,
+            _phantom: Default::default(),
         }
     }
 }
 
-impl<'a> Decimal128Array {
-    /// Constructs a new iterator that iterates `Decimal128` values as i128 values.
-    /// This is kept mostly for back-compatibility purpose.
-    /// Suggests to use `iter()` that returns `Decimal128Iter`.
-    #[allow(deprecated)]
-    pub fn i128_iter(&'a self) -> DecimalIter<'a> {
-        DecimalIter::<'a>::new(self)
-    }
-}
-
-fn build_decimal_array_from<const BYTE_WIDTH: usize>(
+fn build_decimal_array_from<T: DecimalType>(
     null_buf: BooleanBufferBuilder,
     buffer: Buffer,
-) -> BasicDecimalArray<BYTE_WIDTH> {
+) -> DecimalArray<T> {
     let data = unsafe {
         ArrayData::new_unchecked(
-            BasicDecimalArray::<BYTE_WIDTH>::default_type(),
+            DecimalArray::<T>::default_type(),
             null_buf.len(),
             None,
             Some(null_buf.into()),
@@ -423,7 +421,7 @@ fn build_decimal_array_from<const BYTE_WIDTH: usize>(
             vec![],
         )
     };
-    BasicDecimalArray::<BYTE_WIDTH>::from(data)
+    DecimalArray::from(data)
 }
 
 impl<Ptr: Into<Decimal256>> FromIterator<Option<Ptr>> for Decimal256Array {
@@ -446,7 +444,7 @@ impl<Ptr: Into<Decimal256>> FromIterator<Option<Ptr>> for Decimal256Array {
             }
         });
 
-        build_decimal_array_from::<32>(null_buf, buffer.into())
+        build_decimal_array_from(null_buf, buffer.into())
     }
 }
 
@@ -471,11 +469,11 @@ impl<Ptr: Into<i128>> FromIterator<Option<Ptr>> for Decimal128Array {
             })
             .collect();
 
-        build_decimal_array_from::<16>(null_buf, buffer)
+        build_decimal_array_from(null_buf, buffer)
     }
 }
 
-impl<const BYTE_WIDTH: usize> Array for BasicDecimalArray<BYTE_WIDTH> {
+impl<T: DecimalType> Array for DecimalArray<T> {
     fn as_any(&self) -> &dyn Any {
         self
     }
@@ -489,18 +487,18 @@ impl<const BYTE_WIDTH: usize> Array for BasicDecimalArray<BYTE_WIDTH> {
     }
 }
 
-impl<const BYTE_WIDTH: usize> From<BasicDecimalArray<BYTE_WIDTH>> for ArrayData {
-    fn from(array: BasicDecimalArray<BYTE_WIDTH>) -> Self {
+impl<T: DecimalType> From<DecimalArray<T>> for ArrayData {
+    fn from(array: DecimalArray<T>) -> Self {
         array.data
     }
 }
 
-impl<const BYTE_WIDTH: usize> fmt::Debug for BasicDecimalArray<BYTE_WIDTH> {
+impl<T: DecimalType> fmt::Debug for DecimalArray<T> {
     fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
         write!(
             f,
             "Decimal{}Array<{}, {}>\n[\n",
-            BYTE_WIDTH * 8,
+            T::BYTE_LENGTH * 8,
             self.precision,
             self.scale
         )?;
@@ -513,31 +511,31 @@ impl<const BYTE_WIDTH: usize> fmt::Debug for BasicDecimalArray<BYTE_WIDTH> {
     }
 }
 
-impl<'a, const BYTE_WIDTH: usize> ArrayAccessor for &'a BasicDecimalArray<BYTE_WIDTH> {
-    type Item = BasicDecimal<BYTE_WIDTH>;
+impl<'a, T: DecimalType> ArrayAccessor for &'a DecimalArray<T> {
+    type Item = Decimal<T>;
 
     fn value(&self, index: usize) -> Self::Item {
-        BasicDecimalArray::<BYTE_WIDTH>::value(self, index)
+        DecimalArray::<T>::value(self, index)
     }
 
     unsafe fn value_unchecked(&self, index: usize) -> Self::Item {
-        BasicDecimalArray::<BYTE_WIDTH>::value_unchecked(self, index)
+        DecimalArray::<T>::value_unchecked(self, index)
     }
 }
 
-impl<'a, const BYTE_WIDTH: usize> IntoIterator for &'a BasicDecimalArray<BYTE_WIDTH> {
-    type Item = Option<BasicDecimal<BYTE_WIDTH>>;
-    type IntoIter = BasicDecimalIter<'a, BYTE_WIDTH>;
+impl<'a, T: DecimalType> IntoIterator for &'a DecimalArray<T> {
+    type Item = Option<Decimal<T>>;
+    type IntoIter = DecimalIter<'a, T>;
 
     fn into_iter(self) -> Self::IntoIter {
-        BasicDecimalIter::<'a, BYTE_WIDTH>::new(self)
+        DecimalIter::<'a, T>::new(self)
     }
 }
 
-impl<'a, const BYTE_WIDTH: usize> BasicDecimalArray<BYTE_WIDTH> {
+impl<'a, T: DecimalType> DecimalArray<T> {
     /// constructs a new iterator
-    pub fn iter(&'a self) -> BasicDecimalIter<'a, BYTE_WIDTH> {
-        BasicDecimalIter::<'a, BYTE_WIDTH>::new(self)
+    pub fn iter(&'a self) -> DecimalIter<'a, T> {
+        DecimalIter::<'a, T>::new(self)
     }
 }
 
diff --git a/arrow/src/array/builder/decimal_builder.rs b/arrow/src/array/builder/decimal_builder.rs
index 9a76a31dd..7021dce37 100644
--- a/arrow/src/array/builder/decimal_builder.rs
+++ b/arrow/src/array/builder/decimal_builder.rs
@@ -365,7 +365,7 @@ mod tests {
             .expect("should not validate invalid value at builder");
 
         let array = builder.finish();
-        let array_data = array_decimal::BasicDecimalArray::data(&array);
+        let array_data = array_decimal::DecimalArray::data(&array);
         array_data.validate_values().unwrap();
     }
 }
diff --git a/arrow/src/array/iterator.rs b/arrow/src/array/iterator.rs
index 7cc9bde6b..ec93aa265 100644
--- a/arrow/src/array/iterator.rs
+++ b/arrow/src/array/iterator.rs
@@ -16,11 +16,12 @@
 // under the License.
 
 use crate::array::array::ArrayAccessor;
-use crate::array::BasicDecimalArray;
+use crate::array::DecimalArray;
+use crate::datatypes::{Decimal128Type, Decimal256Type};
 
 use super::{
-    Array, BooleanArray, Decimal128Array, GenericBinaryArray, GenericListArray,
-    GenericStringArray, PrimitiveArray,
+    BooleanArray, GenericBinaryArray, GenericListArray, GenericStringArray,
+    PrimitiveArray,
 };
 
 /// an iterator that returns Some(T) or None, that can be used on any [`ArrayAccessor`]
@@ -104,69 +105,14 @@ pub type GenericStringIter<'a, T> = ArrayIter<&'a GenericStringArray<T>>;
 pub type GenericBinaryIter<'a, T> = ArrayIter<&'a GenericBinaryArray<T>>;
 pub type GenericListArrayIter<'a, O> = ArrayIter<&'a GenericListArray<O>>;
 
-pub type BasicDecimalIter<'a, const BYTE_WIDTH: usize> =
-    ArrayIter<&'a BasicDecimalArray<BYTE_WIDTH>>;
+pub type DecimalIter<'a, T> = ArrayIter<&'a DecimalArray<T>>;
 /// an iterator that returns `Some(Decimal128)` or `None`, that can be used on a
-/// [`Decimal128Array`]
-pub type Decimal128Iter<'a> = BasicDecimalIter<'a, 16>;
+/// [`super::Decimal128Array`]
+pub type Decimal128Iter<'a> = DecimalIter<'a, Decimal128Type>;
 
 /// an iterator that returns `Some(Decimal256)` or `None`, that can be used on a
 /// [`super::Decimal256Array`]
-pub type Decimal256Iter<'a> = BasicDecimalIter<'a, 32>;
-/// an iterator that returns `Some(i128)` or `None`, that can be used on a
-/// [`Decimal128Array`]
-#[derive(Debug)]
-#[deprecated(note = "Please use `Decimal128Iter` instead. \
-    `DecimalIter` iterates `Decimal128` values as i128 values. \
-    This is kept mostly for back-compatibility purpose. Suggests to use `Decimal128Array.iter()` \
-    that returns `Decimal128Iter`.")]
-pub struct DecimalIter<'a> {
-    array: &'a Decimal128Array,
-    current: usize,
-    current_end: usize,
-}
-
-#[allow(deprecated)]
-impl<'a> DecimalIter<'a> {
-    pub fn new(array: &'a Decimal128Array) -> Self {
-        Self {
-            array,
-            current: 0,
-            current_end: array.len(),
-        }
-    }
-}
-
-#[allow(deprecated)]
-impl<'a> std::iter::Iterator for DecimalIter<'a> {
-    type Item = Option<i128>;
-
-    fn next(&mut self) -> Option<Self::Item> {
-        if self.current == self.current_end {
-            None
-        } else {
-            let old = self.current;
-            self.current += 1;
-            // TODO: Improve performance by avoiding bounds check here
-            // (by using adding a `value_unchecked, for example)
-            if self.array.is_null(old) {
-                Some(None)
-            } else {
-                Some(Some(self.array.value(old).as_i128()))
-            }
-        }
-    }
-
-    #[inline]
-    fn size_hint(&self) -> (usize, Option<usize>) {
-        let remain = self.current_end - self.current;
-        (remain, Some(remain))
-    }
-}
-
-/// iterator has known size.
-#[allow(deprecated)]
-impl<'a> std::iter::ExactSizeIterator for DecimalIter<'a> {}
+pub type Decimal256Iter<'a> = DecimalIter<'a, Decimal256Type>;
 
 #[cfg(test)]
 mod tests {
diff --git a/arrow/src/array/mod.rs b/arrow/src/array/mod.rs
index 4a7667741..3496ccd88 100644
--- a/arrow/src/array/mod.rs
+++ b/arrow/src/array/mod.rs
@@ -199,15 +199,12 @@ pub(crate) use self::data::BufferSpec;
 pub use self::array_binary::BinaryArray;
 pub use self::array_binary::LargeBinaryArray;
 pub use self::array_boolean::BooleanArray;
-pub use self::array_decimal::BasicDecimalArray;
 pub use self::array_decimal::Decimal128Array;
 pub use self::array_decimal::Decimal256Array;
+pub use self::array_decimal::DecimalArray;
 pub use self::array_fixed_size_binary::FixedSizeBinaryArray;
 pub use self::array_fixed_size_list::FixedSizeListArray;
 
-#[deprecated(note = "Please use `Decimal128Array` instead")]
-pub type DecimalArray = Decimal128Array;
-
 pub use self::array_dictionary::{DictionaryArray, TypedDictionaryArray};
 pub use self::array_list::LargeListArray;
 pub use self::array_list::ListArray;
diff --git a/arrow/src/datatypes/types.rs b/arrow/src/datatypes/types.rs
index 8c0ac7b3f..60b9f31d2 100644
--- a/arrow/src/datatypes/types.rs
+++ b/arrow/src/datatypes/types.rs
@@ -17,6 +17,10 @@
 
 use super::{ArrowPrimitiveType, DataType, IntervalUnit, TimeUnit};
 use crate::datatypes::delta::shift_months;
+use crate::datatypes::{
+    DECIMAL128_MAX_PRECISION, DECIMAL128_MAX_SCALE, DECIMAL256_MAX_PRECISION,
+    DECIMAL256_MAX_SCALE, DECIMAL_DEFAULT_SCALE,
+};
 use chrono::{Duration, NaiveDate};
 use half::f16;
 use std::ops::{Add, Sub};
@@ -455,6 +459,74 @@ impl Date64Type {
     }
 }
 
+mod private {
+    use super::*;
+
+    pub trait DecimalTypeSealed {}
+    impl DecimalTypeSealed for Decimal128Type {}
+    impl DecimalTypeSealed for Decimal256Type {}
+}
+
+/// Trait representing the in-memory layout of a decimal type
+pub trait NativeDecimalType: Send + Sync + Copy + AsRef<[u8]> {
+    fn from_slice(slice: &[u8]) -> Self;
+}
+
+impl<const N: usize> NativeDecimalType for [u8; N] {
+    fn from_slice(slice: &[u8]) -> Self {
+        slice.try_into().unwrap()
+    }
+}
+
+/// A trait over the decimal types, used by [`DecimalArray`] to provide a generic
+/// implementation across the various decimal types
+///
+/// Implemented by [`Decimal128Type`] and [`Decimal256Type`] for [`Decimal128Array`]
+/// and [`Decimal256Array`] respectively
+///
+/// [`DecimalArray`]: [crate::array::DecimalArray]
+/// [`Decimal128Array`]: [crate::array::Decimal128Array]
+/// [`Decimal256Array`]: [crate::array::Decimal256Array]
+pub trait DecimalType: 'static + Send + Sync + private::DecimalTypeSealed {
+    type Native: NativeDecimalType;
+
+    const BYTE_LENGTH: usize;
+    const MAX_PRECISION: usize;
+    const MAX_SCALE: usize;
+    const TYPE_CONSTRUCTOR: fn(usize, usize) -> DataType;
+    const DEFAULT_TYPE: DataType;
+}
+
+/// The decimal type for a Decimal128Array
+#[derive(Debug)]
+pub struct Decimal128Type {}
+
+impl DecimalType for Decimal128Type {
+    type Native = [u8; 16];
+
+    const BYTE_LENGTH: usize = 16;
+    const MAX_PRECISION: usize = DECIMAL128_MAX_PRECISION;
+    const MAX_SCALE: usize = DECIMAL128_MAX_SCALE;
+    const TYPE_CONSTRUCTOR: fn(usize, usize) -> DataType = DataType::Decimal128;
+    const DEFAULT_TYPE: DataType =
+        DataType::Decimal128(DECIMAL128_MAX_PRECISION, DECIMAL_DEFAULT_SCALE);
+}
+
+/// The decimal type for a Decimal256Array
+#[derive(Debug)]
+pub struct Decimal256Type {}
+
+impl DecimalType for Decimal256Type {
+    type Native = [u8; 32];
+
+    const BYTE_LENGTH: usize = 32;
+    const MAX_PRECISION: usize = DECIMAL256_MAX_PRECISION;
+    const MAX_SCALE: usize = DECIMAL256_MAX_SCALE;
+    const TYPE_CONSTRUCTOR: fn(usize, usize) -> DataType = DataType::Decimal256;
+    const DEFAULT_TYPE: DataType =
+        DataType::Decimal256(DECIMAL256_MAX_PRECISION, DECIMAL_DEFAULT_SCALE);
+}
+
 #[cfg(test)]
 mod tests {
     use super::*;
diff --git a/arrow/src/util/decimal.rs b/arrow/src/util/decimal.rs
index 2e5cddc87..f4430a2dc 100644
--- a/arrow/src/util/decimal.rs
+++ b/arrow/src/util/decimal.rs
@@ -18,50 +18,57 @@
 //! Decimal related utils
 
 use crate::datatypes::{
-    DataType, DECIMAL128_MAX_PRECISION, DECIMAL128_MAX_SCALE, DECIMAL256_MAX_PRECISION,
-    DECIMAL256_MAX_SCALE, DECIMAL_DEFAULT_SCALE,
+    DataType, Decimal128Type, Decimal256Type, DecimalType, DECIMAL256_MAX_PRECISION,
+    DECIMAL_DEFAULT_SCALE,
 };
 use crate::error::{ArrowError, Result};
 use num::bigint::BigInt;
 use num::Signed;
 use std::cmp::{min, Ordering};
 
-#[derive(Debug, Clone, Copy)]
-pub struct BasicDecimal<const BYTE_WIDTH: usize> {
+/// [`Decimal`] is the generic representation of a single decimal value
+///
+/// See [`Decimal128`] and [`Decimal256`] for the value types of [`Decimal128Array`]
+/// and [`Decimal256Array`] respectively
+///
+/// [`Decimal128Array`]: [crate::array::Decimal128Array]
+/// [`Decimal256Array`]: [crate::array::Decimal256Array]
+pub struct Decimal<T: DecimalType> {
     precision: usize,
     scale: usize,
-    value: [u8; BYTE_WIDTH],
+    value: T::Native,
 }
 
-impl<const BYTE_WIDTH: usize> BasicDecimal<BYTE_WIDTH> {
-    #[allow(clippy::type_complexity)]
-    const MAX_PRECISION_SCALE_CONSTRUCTOR_DEFAULT_TYPE: (
-        usize,
-        usize,
-        fn(usize, usize) -> DataType,
-        DataType,
-    ) = match BYTE_WIDTH {
-        16 => (
-            DECIMAL128_MAX_PRECISION,
-            DECIMAL128_MAX_SCALE,
-            DataType::Decimal128,
-            DataType::Decimal128(DECIMAL128_MAX_PRECISION, DECIMAL_DEFAULT_SCALE),
-        ),
-        32 => (
-            DECIMAL256_MAX_PRECISION,
-            DECIMAL256_MAX_SCALE,
-            DataType::Decimal256,
-            DataType::Decimal256(DECIMAL256_MAX_PRECISION, DECIMAL_DEFAULT_SCALE),
-        ),
-        _ => panic!("invalid byte width"),
-    };
-
-    pub const MAX_PRECISION: usize = Self::MAX_PRECISION_SCALE_CONSTRUCTOR_DEFAULT_TYPE.0;
-    pub const MAX_SCALE: usize = Self::MAX_PRECISION_SCALE_CONSTRUCTOR_DEFAULT_TYPE.1;
-    pub const TYPE_CONSTRUCTOR: fn(usize, usize) -> DataType =
-        Self::MAX_PRECISION_SCALE_CONSTRUCTOR_DEFAULT_TYPE.2;
-    pub const DEFAULT_TYPE: DataType =
-        Self::MAX_PRECISION_SCALE_CONSTRUCTOR_DEFAULT_TYPE.3;
+/// Manually implement to avoid `T: Debug` bound
+impl<T: DecimalType> std::fmt::Debug for Decimal<T> {
+    fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
+        f.debug_struct("Decimal")
+            .field("scale", &self.precision)
+            .field("precision", &self.precision)
+            // TODO: Could format this better
+            .field("value", &self.value.as_ref())
+            .finish()
+    }
+}
+
+/// Manually implement to avoid `T: Debug` bound
+impl<T: DecimalType> Clone for Decimal<T> {
+    fn clone(&self) -> Self {
+        Self {
+            precision: self.precision,
+            scale: self.scale,
+            value: self.value,
+        }
+    }
+}
+
+impl<T: DecimalType> Copy for Decimal<T> {}
+
+impl<T: DecimalType> Decimal<T> {
+    pub const MAX_PRECISION: usize = T::MAX_PRECISION;
+    pub const MAX_SCALE: usize = T::MAX_SCALE;
+    pub const TYPE_CONSTRUCTOR: fn(usize, usize) -> DataType = T::TYPE_CONSTRUCTOR;
+    pub const DEFAULT_TYPE: DataType = T::DEFAULT_TYPE;
 
     /// Tries to create a decimal value from precision, scale and bytes.
     /// The bytes should be stored in little-endian order.
@@ -72,7 +79,7 @@ impl<const BYTE_WIDTH: usize> BasicDecimal<BYTE_WIDTH> {
     pub fn try_new_from_bytes(
         precision: usize,
         scale: usize,
-        bytes: &[u8; BYTE_WIDTH],
+        bytes: &T::Native,
     ) -> Result<Self>
     where
         Self: Sized,
@@ -99,15 +106,7 @@ impl<const BYTE_WIDTH: usize> BasicDecimal<BYTE_WIDTH> {
             )));
         }
 
-        if bytes.len() == BYTE_WIDTH {
-            Ok(Self::new(precision, scale, bytes))
-        } else {
-            Err(ArrowError::InvalidArgumentError(format!(
-                "Input to Decimal{} must be {} bytes",
-                BYTE_WIDTH * 8,
-                BYTE_WIDTH
-            )))
-        }
+        Ok(Self::new(precision, scale, bytes))
     }
 
     /// Creates a decimal value from precision, scale, and bytes.
@@ -115,7 +114,7 @@ impl<const BYTE_WIDTH: usize> BasicDecimal<BYTE_WIDTH> {
     /// Safety:
     /// This method doesn't check if the precision and scale are valid.
     /// Use `try_new_from_bytes` for safe constructor.
-    pub fn new(precision: usize, scale: usize, bytes: &[u8; BYTE_WIDTH]) -> Self {
+    pub fn new(precision: usize, scale: usize, bytes: &T::Native) -> Self {
         Self {
             precision,
             scale,
@@ -123,7 +122,7 @@ impl<const BYTE_WIDTH: usize> BasicDecimal<BYTE_WIDTH> {
         }
     }
     /// Returns the raw bytes of the integer representation of the decimal.
-    pub fn raw_value(&self) -> &[u8; BYTE_WIDTH] {
+    pub fn raw_value(&self) -> &T::Native {
         &self.value
     }
 
@@ -143,7 +142,7 @@ impl<const BYTE_WIDTH: usize> BasicDecimal<BYTE_WIDTH> {
     #[allow(clippy::inherent_to_string)]
     pub fn to_string(&self) -> String {
         let raw_bytes = self.raw_value();
-        let integer = BigInt::from_signed_bytes_le(raw_bytes);
+        let integer = BigInt::from_signed_bytes_le(raw_bytes.as_ref());
         let value_str = integer.to_string();
         let (sign, rest) =
             value_str.split_at(if integer >= BigInt::from(0) { 0 } else { 1 });
@@ -163,44 +162,47 @@ impl<const BYTE_WIDTH: usize> BasicDecimal<BYTE_WIDTH> {
     }
 }
 
-impl<const BYTE_WIDTH: usize> PartialOrd for BasicDecimal<BYTE_WIDTH> {
+impl<T: DecimalType> PartialOrd for Decimal<T> {
     fn partial_cmp(&self, other: &Self) -> Option<Ordering> {
         assert_eq!(
             self.scale, other.scale,
             "Cannot compare two Decimals with different scale: {}, {}",
             self.scale, other.scale
         );
-        Some(singed_cmp_le_bytes(&self.value, &other.value))
+        Some(singed_cmp_le_bytes(
+            self.value.as_ref(),
+            other.value.as_ref(),
+        ))
     }
 }
 
-impl<const BYTE_WIDTH: usize> Ord for BasicDecimal<BYTE_WIDTH> {
+impl<T: DecimalType> Ord for Decimal<T> {
     fn cmp(&self, other: &Self) -> Ordering {
         assert_eq!(
             self.scale, other.scale,
             "Cannot compare two Decimals with different scale: {}, {}",
             self.scale, other.scale
         );
-        singed_cmp_le_bytes(&self.value, &other.value)
+        singed_cmp_le_bytes(self.value.as_ref(), other.value.as_ref())
     }
 }
 
-impl<const BYTE_WIDTH: usize> PartialEq<Self> for BasicDecimal<BYTE_WIDTH> {
+impl<T: DecimalType> PartialEq<Self> for Decimal<T> {
     fn eq(&self, other: &Self) -> bool {
         assert_eq!(
             self.scale, other.scale,
             "Cannot compare two Decimals with different scale: {}, {}",
             self.scale, other.scale
         );
-        self.value.eq(&other.value)
+        self.value.as_ref().eq(other.value.as_ref())
     }
 }
 
-impl<const BYTE_WIDTH: usize> Eq for BasicDecimal<BYTE_WIDTH> {}
+impl<T: DecimalType> Eq for Decimal<T> {}
 
 /// Represents a decimal value with precision and scale.
 /// The decimal value could represented by a signed 128-bit integer.
-pub type Decimal128 = BasicDecimal<16>;
+pub type Decimal128 = Decimal<Decimal128Type>;
 
 impl Decimal128 {
     /// Creates `Decimal128` from an `i128` value.
@@ -227,7 +229,7 @@ impl From<Decimal128> for i128 {
 
 /// Represents a decimal value with precision and scale.
 /// The decimal value could be represented by a signed 256-bit integer.
-pub type Decimal256 = BasicDecimal<32>;
+pub type Decimal256 = Decimal<Decimal256Type>;
 
 impl Decimal256 {
     /// Constructs a `Decimal256` value from a `BigInt`.