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/10/24 20:21:14 UTC

[arrow-rs] branch master updated: Add timezone abstraction (#2909)

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 d9fd1d537 Add timezone abstraction (#2909)
d9fd1d537 is described below

commit d9fd1d5373a1b19d806d9faeeca36796bd517e4d
Author: Raphael Taylor-Davies <17...@users.noreply.github.com>
AuthorDate: Tue Oct 25 09:21:09 2022 +1300

    Add timezone abstraction (#2909)
    
    * Add timezone abstraction
    
    * Deprecate instead of remove
    
    * Disallow missing colon in timezone
    
    * RAT
---
 arrow-array/Cargo.toml                   |   1 +
 arrow-array/src/array/primitive_array.rs |  17 --
 arrow-array/src/lib.rs                   |   1 +
 arrow-array/src/temporal_conversions.rs  |  12 +-
 arrow-array/src/timezone.rs              | 325 +++++++++++++++++++++++++++++++
 arrow/Cargo.toml                         |   2 +-
 arrow/src/compute/kernels/cast.rs        |  69 ++-----
 arrow/src/compute/kernels/temporal.rs    | 251 +++++++-----------------
 arrow/src/csv/writer.rs                  |  67 +------
 9 files changed, 437 insertions(+), 308 deletions(-)

diff --git a/arrow-array/Cargo.toml b/arrow-array/Cargo.toml
index accc1d3e6..45765d99f 100644
--- a/arrow-array/Cargo.toml
+++ b/arrow-array/Cargo.toml
@@ -49,6 +49,7 @@ arrow-buffer = { version = "25.0.0", path = "../arrow-buffer" }
 arrow-schema = { version = "25.0.0", path = "../arrow-schema" }
 arrow-data = { version = "25.0.0", path = "../arrow-data" }
 chrono = { version = "0.4", default-features = false, features = ["clock"] }
+chrono-tz = { version = "0.7", optional = true }
 num = { version = "0.4", default-features = false, features = ["std"] }
 half = { version = "2.0", default-features = false }
 hashbrown = { version = "0.12", default-features = false }
diff --git a/arrow-array/src/array/primitive_array.rs b/arrow-array/src/array/primitive_array.rs
index 5a9ffd34c..d979d0d93 100644
--- a/arrow-array/src/array/primitive_array.rs
+++ b/arrow-array/src/array/primitive_array.rs
@@ -1483,23 +1483,6 @@ mod tests {
         assert_eq!(array1, array2);
     }
 
-    #[cfg(feature = "chrono-tz")]
-    #[test]
-    fn test_with_timezone() {
-        use crate::compute::hour;
-        let a: TimestampMicrosecondArray = vec![37800000000, 86339000000].into();
-
-        let b = hour(&a).unwrap();
-        assert_eq!(10, b.value(0));
-        assert_eq!(23, b.value(1));
-
-        let a = a.with_timezone(String::from("America/Los_Angeles"));
-
-        let b = hour(&a).unwrap();
-        assert_eq!(2, b.value(0));
-        assert_eq!(15, b.value(1));
-    }
-
     #[test]
     #[should_panic(
         expected = "Trying to access an element at index 4 from a PrimitiveArray of length 3"
diff --git a/arrow-array/src/lib.rs b/arrow-array/src/lib.rs
index cc963925d..4f015793d 100644
--- a/arrow-array/src/lib.rs
+++ b/arrow-array/src/lib.rs
@@ -170,6 +170,7 @@ mod delta;
 pub mod iterator;
 mod raw_pointer;
 pub mod temporal_conversions;
+pub mod timezone;
 mod trusted_len;
 pub mod types;
 
diff --git a/arrow-array/src/temporal_conversions.rs b/arrow-array/src/temporal_conversions.rs
index 9aae83c8a..8b1064115 100644
--- a/arrow-array/src/temporal_conversions.rs
+++ b/arrow-array/src/temporal_conversions.rs
@@ -17,9 +17,10 @@
 
 //! Conversion methods for dates and times.
 
+use crate::timezone::Tz;
 use crate::ArrowPrimitiveType;
 use arrow_schema::{DataType, TimeUnit};
-use chrono::{Duration, NaiveDate, NaiveDateTime, NaiveTime};
+use chrono::{DateTime, Duration, NaiveDate, NaiveDateTime, NaiveTime, TimeZone, Utc};
 
 /// Number of seconds in a day
 pub const SECONDS_IN_DAY: i64 = 86_400;
@@ -187,6 +188,15 @@ pub fn as_datetime<T: ArrowPrimitiveType>(v: i64) -> Option<NaiveDateTime> {
     }
 }
 
+/// Converts an [`ArrowPrimitiveType`] to [`DateTime<Tz>`]
+pub fn as_datetime_with_timezone<T: ArrowPrimitiveType>(
+    v: i64,
+    tz: Tz,
+) -> Option<DateTime<Tz>> {
+    let naive = as_datetime::<T>(v)?;
+    Some(Utc.from_utc_datetime(&naive).with_timezone(&tz))
+}
+
 /// Converts an [`ArrowPrimitiveType`] to [`NaiveDate`]
 pub fn as_date<T: ArrowPrimitiveType>(v: i64) -> Option<NaiveDate> {
     as_datetime::<T>(v).map(|datetime| datetime.date())
diff --git a/arrow-array/src/timezone.rs b/arrow-array/src/timezone.rs
new file mode 100644
index 000000000..4e60c0c42
--- /dev/null
+++ b/arrow-array/src/timezone.rs
@@ -0,0 +1,325 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you 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.
+
+//! Timezone for timestamp arrays
+
+use arrow_schema::ArrowError;
+use chrono::format::{parse, Parsed, StrftimeItems};
+use chrono::FixedOffset;
+pub use private::{Tz, TzOffset};
+
+/// Parses a fixed offset of the form "+09:00"
+fn parse_fixed_offset(tz: &str) -> Result<FixedOffset, ArrowError> {
+    if tz.len() != 6 {
+        return Err(ArrowError::ParseError(format!(
+            "Invalid timezone \"{}\": Expected format [+-]XX:XX",
+            tz
+        )));
+    }
+
+    let mut parsed = Parsed::new();
+    parse(&mut parsed, tz, StrftimeItems::new("%:z"))
+        .and_then(|_| parsed.to_fixed_offset())
+        .map_err(|e| {
+            ArrowError::ParseError(format!("Invalid timezone \"{}\": {}", tz, e))
+        })
+}
+
+#[cfg(feature = "chrono-tz")]
+mod private {
+    use super::*;
+    use chrono::offset::TimeZone;
+    use chrono::{LocalResult, NaiveDate, NaiveDateTime, Offset};
+    use std::str::FromStr;
+
+    /// An [`Offset`] for [`Tz`]
+    #[derive(Debug, Copy, Clone)]
+    pub struct TzOffset {
+        tz: Tz,
+        offset: FixedOffset,
+    }
+
+    impl std::fmt::Display for TzOffset {
+        fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
+            self.offset.fmt(f)
+        }
+    }
+
+    impl Offset for TzOffset {
+        fn fix(&self) -> FixedOffset {
+            self.offset
+        }
+    }
+
+    /// An Arrow [`TimeZone`]
+    #[derive(Debug, Copy, Clone)]
+    pub struct Tz(TzInner);
+
+    #[derive(Debug, Copy, Clone)]
+    enum TzInner {
+        Timezone(chrono_tz::Tz),
+        Offset(FixedOffset),
+    }
+
+    impl FromStr for Tz {
+        type Err = ArrowError;
+
+        fn from_str(tz: &str) -> Result<Self, Self::Err> {
+            if tz.starts_with('+') || tz.starts_with('-') {
+                Ok(Self(TzInner::Offset(parse_fixed_offset(tz)?)))
+            } else {
+                Ok(Self(TzInner::Timezone(tz.parse().map_err(|e| {
+                    ArrowError::ParseError(format!("Invalid timezone \"{}\": {}", tz, e))
+                })?)))
+            }
+        }
+    }
+
+    macro_rules! tz {
+        ($s:ident, $tz:ident, $b:block) => {
+            match $s.0 {
+                TzInner::Timezone($tz) => $b,
+                TzInner::Offset($tz) => $b,
+            }
+        };
+    }
+
+    impl TimeZone for Tz {
+        type Offset = TzOffset;
+
+        fn from_offset(offset: &Self::Offset) -> Self {
+            offset.tz
+        }
+
+        fn offset_from_local_date(&self, local: &NaiveDate) -> LocalResult<Self::Offset> {
+            tz!(self, tz, {
+                tz.offset_from_local_date(local).map(|x| TzOffset {
+                    tz: *self,
+                    offset: x.fix(),
+                })
+            })
+        }
+
+        fn offset_from_local_datetime(
+            &self,
+            local: &NaiveDateTime,
+        ) -> LocalResult<Self::Offset> {
+            tz!(self, tz, {
+                tz.offset_from_local_datetime(local).map(|x| TzOffset {
+                    tz: *self,
+                    offset: x.fix(),
+                })
+            })
+        }
+
+        fn offset_from_utc_date(&self, utc: &NaiveDate) -> Self::Offset {
+            tz!(self, tz, {
+                TzOffset {
+                    tz: *self,
+                    offset: tz.offset_from_utc_date(utc).fix(),
+                }
+            })
+        }
+
+        fn offset_from_utc_datetime(&self, utc: &NaiveDateTime) -> Self::Offset {
+            tz!(self, tz, {
+                TzOffset {
+                    tz: *self,
+                    offset: tz.offset_from_utc_datetime(utc).fix(),
+                }
+            })
+        }
+    }
+
+    #[cfg(test)]
+    mod tests {
+        use super::*;
+        use chrono::{Timelike, Utc};
+
+        #[test]
+        fn test_with_timezone() {
+            let vals = [
+                Utc.timestamp_millis(37800000),
+                Utc.timestamp_millis(86339000),
+            ];
+
+            assert_eq!(10, vals[0].hour());
+            assert_eq!(23, vals[1].hour());
+
+            let tz: Tz = "America/Los_Angeles".parse().unwrap();
+
+            assert_eq!(2, vals[0].with_timezone(&tz).hour());
+            assert_eq!(15, vals[1].with_timezone(&tz).hour());
+        }
+
+        #[test]
+        fn test_using_chrono_tz_and_utc_naive_date_time() {
+            let sydney_tz = "Australia/Sydney".to_string();
+            let tz: Tz = sydney_tz.parse().unwrap();
+            let sydney_offset_without_dst = FixedOffset::east(10 * 60 * 60);
+            let sydney_offset_with_dst = FixedOffset::east(11 * 60 * 60);
+            // Daylight savings ends
+            // When local daylight time was about to reach
+            // Sunday, 4 April 2021, 3:00:00 am clocks were turned backward 1 hour to
+            // Sunday, 4 April 2021, 2:00:00 am local standard time instead.
+
+            // Daylight savings starts
+            // When local standard time was about to reach
+            // Sunday, 3 October 2021, 2:00:00 am clocks were turned forward 1 hour to
+            // Sunday, 3 October 2021, 3:00:00 am local daylight time instead.
+
+            // Sydney 2021-04-04T02:30:00+11:00 is 2021-04-03T15:30:00Z
+            let utc_just_before_sydney_dst_ends =
+                NaiveDate::from_ymd(2021, 4, 3).and_hms_nano(15, 30, 0, 0);
+            assert_eq!(
+                tz.offset_from_utc_datetime(&utc_just_before_sydney_dst_ends)
+                    .fix(),
+                sydney_offset_with_dst
+            );
+            // Sydney 2021-04-04T02:30:00+10:00 is 2021-04-03T16:30:00Z
+            let utc_just_after_sydney_dst_ends =
+                NaiveDate::from_ymd(2021, 4, 3).and_hms_nano(16, 30, 0, 0);
+            assert_eq!(
+                tz.offset_from_utc_datetime(&utc_just_after_sydney_dst_ends)
+                    .fix(),
+                sydney_offset_without_dst
+            );
+            // Sydney 2021-10-03T01:30:00+10:00 is 2021-10-02T15:30:00Z
+            let utc_just_before_sydney_dst_starts =
+                NaiveDate::from_ymd(2021, 10, 2).and_hms_nano(15, 30, 0, 0);
+            assert_eq!(
+                tz.offset_from_utc_datetime(&utc_just_before_sydney_dst_starts)
+                    .fix(),
+                sydney_offset_without_dst
+            );
+            // Sydney 2021-04-04T03:30:00+11:00 is 2021-10-02T16:30:00Z
+            let utc_just_after_sydney_dst_starts =
+                NaiveDate::from_ymd(2022, 10, 2).and_hms_nano(16, 30, 0, 0);
+            assert_eq!(
+                tz.offset_from_utc_datetime(&utc_just_after_sydney_dst_starts)
+                    .fix(),
+                sydney_offset_with_dst
+            );
+        }
+    }
+}
+
+#[cfg(not(feature = "chrono-tz"))]
+mod private {
+    use super::*;
+    use chrono::offset::TimeZone;
+    use chrono::{FixedOffset, LocalResult, NaiveDate, NaiveDateTime, Offset};
+    use std::str::FromStr;
+
+    /// An [`Offset`] for [`Tz`]
+    #[derive(Debug, Copy, Clone)]
+    pub struct TzOffset(FixedOffset);
+
+    impl std::fmt::Display for TzOffset {
+        fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
+            self.0.fmt(f)
+        }
+    }
+
+    impl Offset for TzOffset {
+        fn fix(&self) -> FixedOffset {
+            self.0
+        }
+    }
+
+    /// An Arrow [`TimeZone`]
+    #[derive(Debug, Copy, Clone)]
+    pub struct Tz(FixedOffset);
+
+    impl FromStr for Tz {
+        type Err = ArrowError;
+
+        fn from_str(tz: &str) -> Result<Self, Self::Err> {
+            if tz.starts_with('+') || tz.starts_with('-') {
+                Ok(Self(parse_fixed_offset(tz)?))
+            } else {
+                Err(ArrowError::ParseError(format!(
+                    "Invalid timezone \"{}\": only offset based timezones supported without chrono-tz feature",
+                    tz
+                )))
+            }
+        }
+    }
+
+    impl TimeZone for Tz {
+        type Offset = TzOffset;
+
+        fn from_offset(offset: &Self::Offset) -> Self {
+            Self(offset.0)
+        }
+
+        fn offset_from_local_date(&self, local: &NaiveDate) -> LocalResult<Self::Offset> {
+            self.0.offset_from_local_date(local).map(TzOffset)
+        }
+
+        fn offset_from_local_datetime(
+            &self,
+            local: &NaiveDateTime,
+        ) -> LocalResult<Self::Offset> {
+            self.0.offset_from_local_datetime(local).map(TzOffset)
+        }
+
+        fn offset_from_utc_date(&self, utc: &NaiveDate) -> Self::Offset {
+            TzOffset(self.0.offset_from_utc_date(utc).fix())
+        }
+
+        fn offset_from_utc_datetime(&self, utc: &NaiveDateTime) -> Self::Offset {
+            TzOffset(self.0.offset_from_utc_datetime(utc).fix())
+        }
+    }
+}
+
+#[cfg(test)]
+mod tests {
+    use super::*;
+    use chrono::{NaiveDate, Offset, TimeZone};
+
+    #[test]
+    fn test_with_offset() {
+        let t = NaiveDate::from_ymd(2000, 1, 1);
+
+        let tz: Tz = "-00:00".parse().unwrap();
+        assert_eq!(tz.offset_from_utc_date(&t).fix().local_minus_utc(), 0);
+        let tz: Tz = "+00:00".parse().unwrap();
+        assert_eq!(tz.offset_from_utc_date(&t).fix().local_minus_utc(), 0);
+
+        let tz: Tz = "-10:00".parse().unwrap();
+        assert_eq!(
+            tz.offset_from_utc_date(&t).fix().local_minus_utc(),
+            -10 * 60 * 60
+        );
+        let tz: Tz = "+09:00".parse().unwrap();
+        assert_eq!(
+            tz.offset_from_utc_date(&t).fix().local_minus_utc(),
+            9 * 60 * 60
+        );
+
+        let err = "+9:00".parse::<Tz>().unwrap_err().to_string();
+        assert!(err.contains("Invalid timezone"), "{}", err);
+
+        let err = "+09".parse::<Tz>().unwrap_err().to_string();
+        assert!(err.contains("Invalid timezone"), "{}", err);
+
+        let err = "+0900".parse::<Tz>().unwrap_err().to_string();
+        assert!(err.contains("Invalid timezone"), "{}", err);
+    }
+}
diff --git a/arrow/Cargo.toml b/arrow/Cargo.toml
index 7a933360c..4a1668cc0 100644
--- a/arrow/Cargo.toml
+++ b/arrow/Cargo.toml
@@ -62,7 +62,6 @@ lazy_static = { version = "1.4", default-features = false }
 lz4 = { version = "1.23", default-features = false, optional = true }
 packed_simd = { version = "0.3", default-features = false, optional = true, package = "packed_simd_2" }
 chrono = { version = "0.4", default-features = false, features = ["clock"] }
-chrono-tz = { version = "0.7", default-features = false, optional = true }
 flatbuffers = { version = "22.9.2", default-features = false, features = ["thiserror"], optional = true }
 comfy-table = { version = "6.0", optional = true, default-features = false }
 pyo3 = { version = "0.17", default-features = false, optional = true }
@@ -100,6 +99,7 @@ dyn_cmp_dict = []
 # Enable dyn-arithmetic kernels for dictionary arrays
 # Note: this does not impact arithmetic with scalars
 dyn_arith_dict = []
+chrono-tz = ["arrow-array/chrono-tz"]
 
 [dev-dependencies]
 rand = { version = "0.8", default-features = false, features = ["std", "std_rng"] }
diff --git a/arrow/src/compute/kernels/cast.rs b/arrow/src/compute/kernels/cast.rs
index 66a04e91e..090195c16 100644
--- a/arrow/src/compute/kernels/cast.rs
+++ b/arrow/src/compute/kernels/cast.rs
@@ -35,17 +35,14 @@
 //! assert_eq!(7.0, c.value(2));
 //! ```
 
-use chrono::format::strftime::StrftimeItems;
-use chrono::format::{parse, Parsed};
-use chrono::{NaiveDateTime, Timelike};
+use chrono::{DateTime, NaiveDateTime, Timelike};
 use std::str;
 use std::sync::Arc;
 
 use crate::buffer::MutableBuffer;
 use crate::compute::kernels::cast_utils::string_to_timestamp_nanos;
-use crate::compute::kernels::temporal::return_compute_error_with;
+use crate::compute::try_unary;
 use crate::compute::{divide_scalar, multiply_scalar};
-use crate::compute::{try_unary, using_chrono_tz_and_utc_naive_date_time};
 use crate::datatypes::*;
 use crate::error::{ArrowError, Result};
 use crate::temporal_conversions::{
@@ -57,6 +54,8 @@ use crate::{
     buffer::Buffer, util::display::array_value_to_string,
     util::serialization::lexical_to_string,
 };
+use arrow_array::temporal_conversions::as_datetime_with_timezone;
+use arrow_array::timezone::Tz;
 use arrow_buffer::i256;
 use num::cast::AsPrimitive;
 use num::{NumCast, ToPrimitive};
@@ -1669,7 +1668,7 @@ where
     Arc::new(builder.finish())
 }
 
-fn extract_component_from_datatime_array<
+fn extract_component_from_datetime_array<
     A: ArrayAccessor<Item = T::Native>,
     OffsetSize,
     T: ArrowTemporalType,
@@ -1678,55 +1677,25 @@ fn extract_component_from_datatime_array<
     iter: ArrayIter<A>,
     mut builder: GenericStringBuilder<OffsetSize>,
     tz: &str,
-    mut parsed: Parsed,
     op: F,
 ) -> Result<ArrayRef>
 where
     OffsetSize: OffsetSizeTrait,
-    F: Fn(NaiveDateTime) -> String,
+    F: Fn(DateTime<Tz>) -> String,
     i64: From<T::Native>,
 {
-    if (tz.starts_with('+') || tz.starts_with('-')) && !tz.contains(':') {
-        return_compute_error_with!(
-            "Invalid timezone",
-            "Expected format [+-]XX:XX".to_string()
-        )
-    } else {
-        let tz_parse_result = parse(&mut parsed, tz, StrftimeItems::new("%z"));
-        let fixed_offset_from_parsed = match tz_parse_result {
-            Ok(_) => match parsed.to_fixed_offset() {
-                Ok(fo) => Some(fo),
-                err => return_compute_error_with!("Invalid timezone", err),
-            },
-            _ => None,
-        };
-
-        for value in iter {
-            if let Some(value) = value {
-                match as_datetime::<T>(<i64 as From<_>>::from(value)) {
-                    Some(utc) => {
-                        let fixed_offset = match fixed_offset_from_parsed {
-                            Some(fo) => fo,
-                            None => {
-                                match using_chrono_tz_and_utc_naive_date_time(tz, utc) {
-                                    Some(fo) => fo,
-                                    err => return_compute_error_with!(
-                                        "Unable to parse timezone",
-                                        err
-                                    ),
-                                }
-                            }
-                        };
-                        builder.append_value(op(utc + fixed_offset));
-                    }
-                    err => return_compute_error_with!(
-                        "Unable to read value as datetime",
-                        err
-                    ),
+    let tz: Tz = tz.parse()?;
+    for value in iter {
+        match value {
+            Some(value) => match as_datetime_with_timezone::<T>(value.into(), tz) {
+                Some(time) => builder.append_value(op(time)),
+                _ => {
+                    return Err(ArrowError::ComputeError(
+                        "Unable to read value as datetime".to_string(),
+                    ))
                 }
-            } else {
-                builder.append_null();
-            }
+            },
+            None => builder.append_null(),
         }
     }
     Ok(Arc::new(builder.finish()))
@@ -1747,16 +1716,14 @@ where
     let builder = GenericStringBuilder::<OffsetSize>::new();
 
     if let Some(tz) = tz {
-        let scratch = Parsed::new();
         // The macro calls `as_datetime` on timestamp values of the array.
         // After applying timezone offset on the datatime, calling `to_string` to get
         // the strings.
         let iter = ArrayIter::new(array);
-        extract_component_from_datatime_array::<_, OffsetSize, T, _>(
+        extract_component_from_datetime_array::<_, OffsetSize, T, _>(
             iter,
             builder,
             tz,
-            scratch,
             |t| t.to_string(),
         )
     } else {
diff --git a/arrow/src/compute/kernels/temporal.rs b/arrow/src/compute/kernels/temporal.rs
index 220b7dadc..54799a326 100644
--- a/arrow/src/compute/kernels/temporal.rs
+++ b/arrow/src/compute/kernels/temporal.rs
@@ -17,16 +17,16 @@
 
 //! Defines temporal kernels for time and date related functions.
 
-use chrono::{Datelike, NaiveDateTime, NaiveTime, Timelike};
+use chrono::{DateTime, Datelike, NaiveDateTime, NaiveTime, Offset, Timelike};
 
 use crate::array::*;
 use crate::datatypes::*;
 use crate::error::{ArrowError, Result};
-use arrow_array::temporal_conversions::{as_datetime, as_time};
+use arrow_array::temporal_conversions::{
+    as_datetime, as_datetime_with_timezone, as_time,
+};
 
-use chrono::format::strftime::StrftimeItems;
-use chrono::format::{parse, Parsed};
-use chrono::FixedOffset;
+use arrow_array::timezone::Tz;
 
 /// This function takes an `ArrayIter` of input array and an extractor `op` which takes
 /// an input `NaiveTime` and returns time component (e.g. hour) as `i32` value.
@@ -87,7 +87,7 @@ where
 /// object used to parse timezone string. `op` is the extractor closure which takes
 /// data time object of `NaiveDateTime` type and returns `i32` value of extracted
 /// component.
-fn extract_component_from_datatime_array<
+fn extract_component_from_datetime_array<
     A: ArrayAccessor<Item = T::Native>,
     T: ArrowTemporalType,
     F,
@@ -95,54 +95,24 @@ fn extract_component_from_datatime_array<
     iter: ArrayIter<A>,
     mut builder: PrimitiveBuilder<Int32Type>,
     tz: &str,
-    mut parsed: Parsed,
     op: F,
 ) -> Result<Int32Array>
 where
-    F: Fn(NaiveDateTime) -> i32,
+    F: Fn(DateTime<Tz>) -> i32,
     i64: From<T::Native>,
 {
-    if (tz.starts_with('+') || tz.starts_with('-')) && !tz.contains(':') {
-        return_compute_error_with!(
-            "Invalid timezone",
-            "Expected format [+-]XX:XX".to_string()
-        )
-    } else {
-        let tz_parse_result = parse(&mut parsed, tz, StrftimeItems::new("%z"));
-        let fixed_offset_from_parsed = match tz_parse_result {
-            Ok(_) => match parsed.to_fixed_offset() {
-                Ok(fo) => Some(fo),
-                err => return_compute_error_with!("Invalid timezone", err),
-            },
-            _ => None,
-        };
-
-        for value in iter {
-            if let Some(value) = value {
-                match as_datetime::<T>(i64::from(value)) {
-                    Some(utc) => {
-                        let fixed_offset = match fixed_offset_from_parsed {
-                            Some(fo) => fo,
-                            None => {
-                                match using_chrono_tz_and_utc_naive_date_time(tz, utc) {
-                                    Some(fo) => fo,
-                                    err => return_compute_error_with!(
-                                        "Unable to parse timezone",
-                                        err
-                                    ),
-                                }
-                            }
-                        };
-                        builder.append_value(op(utc + fixed_offset));
-                    }
-                    err => return_compute_error_with!(
-                        "Unable to read value as datetime",
-                        err
-                    ),
+    let tz: Tz = tz.parse()?;
+    for value in iter {
+        match value {
+            Some(value) => match as_datetime_with_timezone::<T>(value.into(), tz) {
+                Some(time) => builder.append_value(op(time)),
+                _ => {
+                    return Err(ArrowError::ComputeError(
+                        "Unable to read value as datetime".to_string(),
+                    ))
                 }
-            } else {
-                builder.append_null();
-            }
+            },
+            None => builder.append_null(),
         }
     }
     Ok(builder.finish())
@@ -189,27 +159,18 @@ impl<T: Datelike> ChronoDateExt for T {
     }
 }
 
-#[cfg(not(feature = "chrono-tz"))]
-pub fn using_chrono_tz_and_utc_naive_date_time(
-    _tz: &str,
-    _utc: chrono::NaiveDateTime,
-) -> Option<FixedOffset> {
-    None
-}
-
 /// Parse the given string into a string representing fixed-offset that is correct as of the given
 /// UTC NaiveDateTime.
 /// 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.
-#[cfg(feature = "chrono-tz")]
+#[deprecated(note = "Use arrow_array::timezone::Tz instead")]
 pub fn using_chrono_tz_and_utc_naive_date_time(
     tz: &str,
-    utc: chrono::NaiveDateTime,
-) -> Option<FixedOffset> {
-    use chrono::{Offset, TimeZone};
-    tz.parse::<chrono_tz::Tz>()
-        .map(|tz| tz.offset_from_utc_datetime(&utc).fix())
-        .ok()
+    utc: NaiveDateTime,
+) -> Option<chrono::offset::FixedOffset> {
+    use chrono::TimeZone;
+    let tz: Tz = tz.parse().ok()?;
+    Some(tz.offset_from_utc_datetime(&utc).fix())
 }
 
 /// Extracts the hours of a given temporal primitive array as an array of integers within
@@ -217,7 +178,7 @@ pub fn using_chrono_tz_and_utc_naive_date_time(
 pub fn hour<T>(array: &PrimitiveArray<T>) -> Result<Int32Array>
 where
     T: ArrowTemporalType + ArrowNumericType,
-    i64: std::convert::From<T::Native>,
+    i64: From<T::Native>,
 {
     hour_generic::<T, _>(array)
 }
@@ -227,7 +188,7 @@ where
 pub fn hour_generic<T, A: ArrayAccessor<Item = T::Native>>(array: A) -> Result<Int32Array>
 where
     T: ArrowTemporalType + ArrowNumericType,
-    i64: std::convert::From<T::Native>,
+    i64: From<T::Native>,
 {
     match array.data_type().clone() {
         DataType::Dictionary(_, value_type) => {
@@ -244,7 +205,7 @@ fn hour_internal<T, A: ArrayAccessor<Item = T::Native>>(
 ) -> Result<Int32Array>
 where
     T: ArrowTemporalType + ArrowNumericType,
-    i64: std::convert::From<T::Native>,
+    i64: From<T::Native>,
 {
     let b = Int32Builder::with_capacity(array.len());
     match dt {
@@ -257,9 +218,8 @@ where
             Ok(as_datetime_with_op::<A, T, _>(iter, b, |t| t.hour() as i32))
         }
         DataType::Timestamp(_, Some(tz)) => {
-            let scratch = Parsed::new();
             let iter = ArrayIter::new(array);
-            extract_component_from_datatime_array::<A, T, _>(iter, b, tz, scratch, |t| {
+            extract_component_from_datetime_array::<A, T, _>(iter, b, tz, |t| {
                 t.hour() as i32
             })
         }
@@ -271,7 +231,7 @@ where
 pub fn year<T>(array: &PrimitiveArray<T>) -> Result<Int32Array>
 where
     T: ArrowTemporalType + ArrowNumericType,
-    i64: std::convert::From<T::Native>,
+    i64: From<T::Native>,
 {
     year_generic::<T, _>(array)
 }
@@ -280,7 +240,7 @@ where
 pub fn year_generic<T, A: ArrayAccessor<Item = T::Native>>(array: A) -> Result<Int32Array>
 where
     T: ArrowTemporalType + ArrowNumericType,
-    i64: std::convert::From<T::Native>,
+    i64: From<T::Native>,
 {
     match array.data_type().clone() {
         DataType::Dictionary(_, value_type) => {
@@ -297,7 +257,7 @@ fn year_internal<T, A: ArrayAccessor<Item = T::Native>>(
 ) -> Result<Int32Array>
 where
     T: ArrowTemporalType + ArrowNumericType,
-    i64: std::convert::From<T::Native>,
+    i64: From<T::Native>,
 {
     match dt {
         DataType::Date32 | DataType::Date64 | DataType::Timestamp(_, _) => {
@@ -314,7 +274,7 @@ where
 pub fn quarter<T>(array: &PrimitiveArray<T>) -> Result<Int32Array>
 where
     T: ArrowTemporalType + ArrowNumericType,
-    i64: std::convert::From<T::Native>,
+    i64: From<T::Native>,
 {
     quarter_generic::<T, _>(array)
 }
@@ -326,7 +286,7 @@ pub fn quarter_generic<T, A: ArrayAccessor<Item = T::Native>>(
 ) -> Result<Int32Array>
 where
     T: ArrowTemporalType + ArrowNumericType,
-    i64: std::convert::From<T::Native>,
+    i64: From<T::Native>,
 {
     match array.data_type().clone() {
         DataType::Dictionary(_, value_type) => {
@@ -343,7 +303,7 @@ fn quarter_internal<T, A: ArrayAccessor<Item = T::Native>>(
 ) -> Result<Int32Array>
 where
     T: ArrowTemporalType + ArrowNumericType,
-    i64: std::convert::From<T::Native>,
+    i64: From<T::Native>,
 {
     let b = Int32Builder::with_capacity(array.len());
     match dt {
@@ -354,9 +314,8 @@ where
             }))
         }
         DataType::Timestamp(_, Some(tz)) => {
-            let scratch = Parsed::new();
             let iter = ArrayIter::new(array);
-            extract_component_from_datatime_array::<A, T, _>(iter, b, tz, scratch, |t| {
+            extract_component_from_datetime_array::<A, T, _>(iter, b, tz, |t| {
                 t.quarter() as i32
             })
         }
@@ -369,7 +328,7 @@ where
 pub fn month<T>(array: &PrimitiveArray<T>) -> Result<Int32Array>
 where
     T: ArrowTemporalType + ArrowNumericType,
-    i64: std::convert::From<T::Native>,
+    i64: From<T::Native>,
 {
     month_generic::<T, _>(array)
 }
@@ -380,7 +339,7 @@ pub fn month_generic<T, A: ArrayAccessor<Item = T::Native>>(
 ) -> Result<Int32Array>
 where
     T: ArrowTemporalType + ArrowNumericType,
-    i64: std::convert::From<T::Native>,
+    i64: From<T::Native>,
 {
     match array.data_type().clone() {
         DataType::Dictionary(_, value_type) => {
@@ -397,7 +356,7 @@ fn month_internal<T, A: ArrayAccessor<Item = T::Native>>(
 ) -> Result<Int32Array>
 where
     T: ArrowTemporalType + ArrowNumericType,
-    i64: std::convert::From<T::Native>,
+    i64: From<T::Native>,
 {
     let b = Int32Builder::with_capacity(array.len());
     match dt {
@@ -408,9 +367,8 @@ where
             }))
         }
         DataType::Timestamp(_, Some(tz)) => {
-            let scratch = Parsed::new();
             let iter = ArrayIter::new(array);
-            extract_component_from_datatime_array::<A, T, _>(iter, b, tz, scratch, |t| {
+            extract_component_from_datetime_array::<A, T, _>(iter, b, tz, |t| {
                 t.month() as i32
             })
         }
@@ -427,7 +385,7 @@ where
 pub fn num_days_from_monday<T>(array: &PrimitiveArray<T>) -> Result<Int32Array>
 where
     T: ArrowTemporalType + ArrowNumericType,
-    i64: std::convert::From<T::Native>,
+    i64: From<T::Native>,
 {
     num_days_from_monday_generic::<T, _>(array)
 }
@@ -443,7 +401,7 @@ pub fn num_days_from_monday_generic<T, A: ArrayAccessor<Item = T::Native>>(
 ) -> Result<Int32Array>
 where
     T: ArrowTemporalType + ArrowNumericType,
-    i64: std::convert::From<T::Native>,
+    i64: From<T::Native>,
 {
     match array.data_type().clone() {
         DataType::Dictionary(_, value_type) => {
@@ -465,7 +423,7 @@ fn num_days_from_monday_internal<T, A: ArrayAccessor<Item = T::Native>>(
 ) -> Result<Int32Array>
 where
     T: ArrowTemporalType + ArrowNumericType,
-    i64: std::convert::From<T::Native>,
+    i64: From<T::Native>,
 {
     let b = Int32Builder::with_capacity(array.len());
     match dt {
@@ -476,9 +434,8 @@ where
             }))
         }
         DataType::Timestamp(_, Some(tz)) => {
-            let scratch = Parsed::new();
             let iter = ArrayIter::new(array);
-            extract_component_from_datatime_array::<A, T, _>(iter, b, tz, scratch, |t| {
+            extract_component_from_datetime_array::<A, T, _>(iter, b, tz, |t| {
                 t.num_days_from_monday()
             })
         }
@@ -495,7 +452,7 @@ where
 pub fn num_days_from_sunday<T>(array: &PrimitiveArray<T>) -> Result<Int32Array>
 where
     T: ArrowTemporalType + ArrowNumericType,
-    i64: std::convert::From<T::Native>,
+    i64: From<T::Native>,
 {
     num_days_from_sunday_generic::<T, _>(array)
 }
@@ -511,7 +468,7 @@ pub fn num_days_from_sunday_generic<T, A: ArrayAccessor<Item = T::Native>>(
 ) -> Result<Int32Array>
 where
     T: ArrowTemporalType + ArrowNumericType,
-    i64: std::convert::From<T::Native>,
+    i64: From<T::Native>,
 {
     match array.data_type().clone() {
         DataType::Dictionary(_, value_type) => {
@@ -533,7 +490,7 @@ fn num_days_from_sunday_internal<T, A: ArrayAccessor<Item = T::Native>>(
 ) -> Result<Int32Array>
 where
     T: ArrowTemporalType + ArrowNumericType,
-    i64: std::convert::From<T::Native>,
+    i64: From<T::Native>,
 {
     let b = Int32Builder::with_capacity(array.len());
     match dt {
@@ -544,9 +501,8 @@ where
             }))
         }
         DataType::Timestamp(_, Some(tz)) => {
-            let scratch = Parsed::new();
             let iter = ArrayIter::new(array);
-            extract_component_from_datatime_array::<A, T, _>(iter, b, tz, scratch, |t| {
+            extract_component_from_datetime_array::<A, T, _>(iter, b, tz, |t| {
                 t.num_days_from_sunday()
             })
         }
@@ -561,7 +517,7 @@ where
 pub fn day<T>(array: &PrimitiveArray<T>) -> Result<Int32Array>
 where
     T: ArrowTemporalType + ArrowNumericType,
-    i64: std::convert::From<T::Native>,
+    i64: From<T::Native>,
 {
     day_generic::<T, _>(array)
 }
@@ -570,7 +526,7 @@ where
 pub fn day_generic<T, A: ArrayAccessor<Item = T::Native>>(array: A) -> Result<Int32Array>
 where
     T: ArrowTemporalType + ArrowNumericType,
-    i64: std::convert::From<T::Native>,
+    i64: From<T::Native>,
 {
     match array.data_type().clone() {
         DataType::Dictionary(_, value_type) => {
@@ -587,7 +543,7 @@ fn day_internal<T, A: ArrayAccessor<Item = T::Native>>(
 ) -> Result<Int32Array>
 where
     T: ArrowTemporalType + ArrowNumericType,
-    i64: std::convert::From<T::Native>,
+    i64: From<T::Native>,
 {
     let b = Int32Builder::with_capacity(array.len());
     match dt {
@@ -596,9 +552,8 @@ where
             Ok(as_datetime_with_op::<A, T, _>(iter, b, |t| t.day() as i32))
         }
         DataType::Timestamp(_, Some(ref tz)) => {
-            let scratch = Parsed::new();
             let iter = ArrayIter::new(array);
-            extract_component_from_datatime_array::<A, T, _>(iter, b, tz, scratch, |t| {
+            extract_component_from_datetime_array::<A, T, _>(iter, b, tz, |t| {
                 t.day() as i32
             })
         }
@@ -611,7 +566,7 @@ where
 pub fn doy<T>(array: &PrimitiveArray<T>) -> Result<Int32Array>
 where
     T: ArrowTemporalType + ArrowNumericType,
-    i64: std::convert::From<T::Native>,
+    i64: From<T::Native>,
 {
     doy_generic::<T, _>(array)
 }
@@ -621,7 +576,7 @@ where
 pub fn doy_generic<T, A: ArrayAccessor<Item = T::Native>>(array: A) -> Result<Int32Array>
 where
     T: ArrowTemporalType + ArrowNumericType,
-    i64: std::convert::From<T::Native>,
+    i64: From<T::Native>,
 {
     match array.data_type().clone() {
         DataType::Dictionary(_, value_type) => {
@@ -640,7 +595,7 @@ fn doy_internal<T, A: ArrayAccessor<Item = T::Native>>(
 where
     T: ArrowTemporalType + ArrowNumericType,
     T::Native: ArrowNativeType,
-    i64: std::convert::From<T::Native>,
+    i64: From<T::Native>,
 {
     let b = Int32Builder::with_capacity(array.len());
     match dt {
@@ -651,9 +606,8 @@ where
             }))
         }
         DataType::Timestamp(_, Some(ref tz)) => {
-            let scratch = Parsed::new();
             let iter = ArrayIter::new(array);
-            extract_component_from_datatime_array::<A, T, _>(iter, b, tz, scratch, |t| {
+            extract_component_from_datetime_array::<A, T, _>(iter, b, tz, |t| {
                 t.ordinal() as i32
             })
         }
@@ -665,7 +619,7 @@ where
 pub fn minute<T>(array: &PrimitiveArray<T>) -> Result<Int32Array>
 where
     T: ArrowTemporalType + ArrowNumericType,
-    i64: std::convert::From<T::Native>,
+    i64: From<T::Native>,
 {
     minute_generic::<T, _>(array)
 }
@@ -676,7 +630,7 @@ pub fn minute_generic<T, A: ArrayAccessor<Item = T::Native>>(
 ) -> Result<Int32Array>
 where
     T: ArrowTemporalType + ArrowNumericType,
-    i64: std::convert::From<T::Native>,
+    i64: From<T::Native>,
 {
     match array.data_type().clone() {
         DataType::Dictionary(_, value_type) => {
@@ -693,7 +647,7 @@ fn minute_internal<T, A: ArrayAccessor<Item = T::Native>>(
 ) -> Result<Int32Array>
 where
     T: ArrowTemporalType + ArrowNumericType,
-    i64: std::convert::From<T::Native>,
+    i64: From<T::Native>,
 {
     let b = Int32Builder::with_capacity(array.len());
     match dt {
@@ -704,9 +658,8 @@ where
             }))
         }
         DataType::Timestamp(_, Some(tz)) => {
-            let scratch = Parsed::new();
             let iter = ArrayIter::new(array);
-            extract_component_from_datatime_array::<A, T, _>(iter, b, tz, scratch, |t| {
+            extract_component_from_datetime_array::<A, T, _>(iter, b, tz, |t| {
                 t.minute() as i32
             })
         }
@@ -718,7 +671,7 @@ where
 pub fn week<T>(array: &PrimitiveArray<T>) -> Result<Int32Array>
 where
     T: ArrowTemporalType + ArrowNumericType,
-    i64: std::convert::From<T::Native>,
+    i64: From<T::Native>,
 {
     week_generic::<T, _>(array)
 }
@@ -727,7 +680,7 @@ where
 pub fn week_generic<T, A: ArrayAccessor<Item = T::Native>>(array: A) -> Result<Int32Array>
 where
     T: ArrowTemporalType + ArrowNumericType,
-    i64: std::convert::From<T::Native>,
+    i64: From<T::Native>,
 {
     match array.data_type().clone() {
         DataType::Dictionary(_, value_type) => {
@@ -744,7 +697,7 @@ fn week_internal<T, A: ArrayAccessor<Item = T::Native>>(
 ) -> Result<Int32Array>
 where
     T: ArrowTemporalType + ArrowNumericType,
-    i64: std::convert::From<T::Native>,
+    i64: From<T::Native>,
 {
     match dt {
         DataType::Date32 | DataType::Date64 | DataType::Timestamp(_, None) => {
@@ -762,7 +715,7 @@ where
 pub fn second<T>(array: &PrimitiveArray<T>) -> Result<Int32Array>
 where
     T: ArrowTemporalType + ArrowNumericType,
-    i64: std::convert::From<T::Native>,
+    i64: From<T::Native>,
 {
     second_generic::<T, _>(array)
 }
@@ -773,7 +726,7 @@ pub fn second_generic<T, A: ArrayAccessor<Item = T::Native>>(
 ) -> Result<Int32Array>
 where
     T: ArrowTemporalType + ArrowNumericType,
-    i64: std::convert::From<T::Native>,
+    i64: From<T::Native>,
 {
     match array.data_type().clone() {
         DataType::Dictionary(_, value_type) => {
@@ -790,7 +743,7 @@ fn second_internal<T, A: ArrayAccessor<Item = T::Native>>(
 ) -> Result<Int32Array>
 where
     T: ArrowTemporalType + ArrowNumericType,
-    i64: std::convert::From<T::Native>,
+    i64: From<T::Native>,
 {
     let b = Int32Builder::with_capacity(array.len());
     match dt {
@@ -801,9 +754,8 @@ where
             }))
         }
         DataType::Timestamp(_, Some(tz)) => {
-            let scratch = Parsed::new();
             let iter = ArrayIter::new(array);
-            extract_component_from_datatime_array::<A, T, _>(iter, b, tz, scratch, |t| {
+            extract_component_from_datetime_array::<A, T, _>(iter, b, tz, |t| {
                 t.second() as i32
             })
         }
@@ -814,8 +766,6 @@ where
 #[cfg(test)]
 mod tests {
     use super::*;
-    #[cfg(feature = "chrono-tz")]
-    use chrono::NaiveDate;
 
     #[test]
     fn test_temporal_array_date64_hour() {
@@ -1180,21 +1130,24 @@ mod tests {
     fn test_temporal_array_timestamp_hour_with_timezone_without_colon() {
         let a =
             TimestampSecondArray::from_vec(vec![60 * 60 * 10], Some("+0100".to_string()));
-        assert!(matches!(hour(&a), Err(ArrowError::ComputeError(_))))
+        let err = hour(&a).unwrap_err().to_string();
+        assert!(err.contains("Invalid timezone"), "{}", err);
     }
 
     #[test]
     fn test_temporal_array_timestamp_hour_with_timezone_without_initial_sign() {
         let a =
             TimestampSecondArray::from_vec(vec![60 * 60 * 10], Some("0100".to_string()));
-        assert!(matches!(hour(&a), Err(ArrowError::ComputeError(_))))
+        let err = hour(&a).unwrap_err().to_string();
+        assert!(err.contains("Invalid timezone"), "{}", err);
     }
 
     #[test]
     fn test_temporal_array_timestamp_hour_with_timezone_with_only_colon() {
         let a =
             TimestampSecondArray::from_vec(vec![60 * 60 * 10], Some("01:00".to_string()));
-        assert!(matches!(hour(&a), Err(ArrowError::ComputeError(_))))
+        let err = hour(&a).unwrap_err().to_string();
+        assert!(err.contains("Invalid timezone"), "{}", err);
     }
 
     #[cfg(feature = "chrono-tz")]
@@ -1231,65 +1184,7 @@ mod tests {
             vec![60 * 60 * 10],
             Some("Asia/Kolkatta".to_string()),
         );
-        assert!(matches!(hour(&a), Err(ArrowError::ComputeError(_))))
-    }
-
-    #[cfg(feature = "chrono-tz")]
-    #[test]
-    fn test_using_chrono_tz_and_utc_naive_date_time() {
-        let sydney_tz = "Australia/Sydney".to_string();
-        let sydney_offset_without_dst = FixedOffset::east(10 * 60 * 60);
-        let sydney_offset_with_dst = FixedOffset::east(11 * 60 * 60);
-        // Daylight savings ends
-        // When local daylight time was about to reach
-        // Sunday, 4 April 2021, 3:00:00 am clocks were turned backward 1 hour to
-        // Sunday, 4 April 2021, 2:00:00 am local standard time instead.
-
-        // Daylight savings starts
-        // When local standard time was about to reach
-        // Sunday, 3 October 2021, 2:00:00 am clocks were turned forward 1 hour to
-        // Sunday, 3 October 2021, 3:00:00 am local daylight time instead.
-
-        // Sydney 2021-04-04T02:30:00+11:00 is 2021-04-03T15:30:00Z
-        let utc_just_before_sydney_dst_ends =
-            NaiveDate::from_ymd(2021, 4, 3).and_hms_nano(15, 30, 0, 0);
-        assert_eq!(
-            using_chrono_tz_and_utc_naive_date_time(
-                &sydney_tz,
-                utc_just_before_sydney_dst_ends
-            ),
-            Some(sydney_offset_with_dst)
-        );
-        // Sydney 2021-04-04T02:30:00+10:00 is 2021-04-03T16:30:00Z
-        let utc_just_after_sydney_dst_ends =
-            NaiveDate::from_ymd(2021, 4, 3).and_hms_nano(16, 30, 0, 0);
-        assert_eq!(
-            using_chrono_tz_and_utc_naive_date_time(
-                &sydney_tz,
-                utc_just_after_sydney_dst_ends
-            ),
-            Some(sydney_offset_without_dst)
-        );
-        // Sydney 2021-10-03T01:30:00+10:00 is 2021-10-02T15:30:00Z
-        let utc_just_before_sydney_dst_starts =
-            NaiveDate::from_ymd(2021, 10, 2).and_hms_nano(15, 30, 0, 0);
-        assert_eq!(
-            using_chrono_tz_and_utc_naive_date_time(
-                &sydney_tz,
-                utc_just_before_sydney_dst_starts
-            ),
-            Some(sydney_offset_without_dst)
-        );
-        // Sydney 2021-04-04T03:30:00+11:00 is 2021-10-02T16:30:00Z
-        let utc_just_after_sydney_dst_starts =
-            NaiveDate::from_ymd(2022, 10, 2).and_hms_nano(16, 30, 0, 0);
-        assert_eq!(
-            using_chrono_tz_and_utc_naive_date_time(
-                &sydney_tz,
-                utc_just_after_sydney_dst_starts
-            ),
-            Some(sydney_offset_with_dst)
-        );
+        assert!(matches!(hour(&a), Err(ArrowError::ParseError(_))))
     }
 
     #[test]
diff --git a/arrow/src/csv/writer.rs b/arrow/src/csv/writer.rs
index eb7a8fd5b..e92e68e5f 100644
--- a/arrow/src/csv/writer.rs
+++ b/arrow/src/csv/writer.rs
@@ -63,12 +63,9 @@
 //! }
 //! ```
 
-use std::io::Write;
-
-#[cfg(feature = "chrono-tz")]
-use crate::compute::kernels::temporal::using_chrono_tz_and_utc_naive_date_time;
-#[cfg(feature = "chrono-tz")]
+use arrow_array::timezone::Tz;
 use chrono::{DateTime, Utc};
+use std::io::Write;
 
 use crate::csv::map_csv_error;
 use crate::datatypes::*;
@@ -239,45 +236,6 @@ impl<W: Write> Writer<W> {
         Ok(())
     }
 
-    #[cfg(not(feature = "chrono-tz"))]
-    fn handle_timestamp(
-        &self,
-        time_unit: &TimeUnit,
-        _time_zone: Option<&String>,
-        row_index: usize,
-        col: &ArrayRef,
-    ) -> Result<String> {
-        use TimeUnit::*;
-        let datetime = match time_unit {
-            Second => col
-                .as_any()
-                .downcast_ref::<TimestampSecondArray>()
-                .unwrap()
-                .value_as_datetime(row_index)
-                .unwrap(),
-            Millisecond => col
-                .as_any()
-                .downcast_ref::<TimestampMillisecondArray>()
-                .unwrap()
-                .value_as_datetime(row_index)
-                .unwrap(),
-            Microsecond => col
-                .as_any()
-                .downcast_ref::<TimestampMicrosecondArray>()
-                .unwrap()
-                .value_as_datetime(row_index)
-                .unwrap(),
-            Nanosecond => col
-                .as_any()
-                .downcast_ref::<TimestampNanosecondArray>()
-                .unwrap()
-                .value_as_datetime(row_index)
-                .unwrap(),
-        };
-        Ok(format!("{}", datetime.format(&self.timestamp_format)))
-    }
-
-    #[cfg(feature = "chrono-tz")]
     fn handle_timestamp(
         &self,
         time_unit: &TimeUnit,
@@ -286,7 +244,6 @@ impl<W: Write> Writer<W> {
         col: &ArrayRef,
     ) -> Result<String> {
         use TimeUnit::*;
-
         let datetime = match time_unit {
             Second => col
                 .as_any()
@@ -313,25 +270,15 @@ impl<W: Write> Writer<W> {
                 .value_as_datetime(row_index)
                 .unwrap(),
         };
-        let tzs = match time_zone {
-            None => "UTC".to_string(),
-            Some(tzs) => tzs.to_string(),
-        };
 
-        match using_chrono_tz_and_utc_naive_date_time(&tzs, datetime) {
+        let tz: Option<Tz> = time_zone.map(|x| x.parse()).transpose()?;
+        match tz {
             Some(tz) => {
                 let utc_time = DateTime::<Utc>::from_utc(datetime, Utc);
-                Ok(format!(
-                    "{}",
-                    utc_time
-                        .with_timezone(&tz)
-                        .format(&self.timestamp_tz_format)
-                ))
+                let local_time = utc_time.with_timezone(&tz);
+                Ok(local_time.format(&self.timestamp_tz_format).to_string())
             }
-            err => Err(ArrowError::ComputeError(format!(
-                "{}: {:?}",
-                "Unable to parse timezone", err
-            ))),
+            None => Ok(datetime.format(&self.timestamp_format).to_string()),
         }
     }