You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by we...@apache.org on 2020/08/16 20:13:14 UTC

[arrow] branch master updated: ARROW-9528: [Python] Honor tzinfo when converting from datetime

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 2e3d7ec  ARROW-9528: [Python] Honor tzinfo when converting from datetime
2e3d7ec is described below

commit 2e3d7ecd320d3e91d285ad0ee729aa18e2b4e476
Author: Krisztián Szűcs <sz...@gmail.com>
AuthorDate: Sun Aug 16 15:12:28 2020 -0500

    ARROW-9528: [Python] Honor tzinfo when converting from datetime
    
    Follow up of:
    - ARROW-9223: [Python] Propagate timezone information in pandas conversion
    - ARROW-9528: [Python] Honor tzinfo when converting from datetime (https://github.com/apache/arrow/pull/7805)
    
    TODOs:
    - [x] Store all Timestamp values normalized to UTC
    - [x] Infer timezone from the array values if no explicit type was given
    - [x] Testing (especially pandas object roundtrip)
    - [x] Testing of timezone-naive roundtrips
    - [x] Testing mixed pandas and datetime objects
    
    Closes #7816 from kszucs/tz
    
    Lead-authored-by: Krisztián Szűcs <sz...@gmail.com>
    Co-authored-by: Micah Kornfield <em...@gmail.com>
    Signed-off-by: Wes McKinney <we...@apache.org>
---
 ci/scripts/integration_spark.sh                |   3 +
 cpp/src/arrow/compute/kernels/scalar_string.cc |   4 +-
 cpp/src/arrow/python/arrow_to_pandas.cc        |  53 ++++--
 cpp/src/arrow/python/arrow_to_pandas.h         |   5 +-
 cpp/src/arrow/python/datetime.cc               | 172 +++++++++++++++++-
 cpp/src/arrow/python/datetime.h                |  26 +++
 cpp/src/arrow/python/inference.cc              |  22 +--
 cpp/src/arrow/python/python_to_arrow.cc        | 151 +++++++++-------
 cpp/src/arrow/python/python_to_arrow.h         |   8 +-
 python/pyarrow/array.pxi                       |   7 +-
 python/pyarrow/includes/libarrow.pxd           |   5 +
 python/pyarrow/tests/test_array.py             |  22 ++-
 python/pyarrow/tests/test_convert_builtin.py   | 234 ++++++++++++++++++++-----
 python/pyarrow/tests/test_pandas.py            |  60 +++++--
 python/pyarrow/tests/test_types.py             | 117 +++++++++++++
 python/pyarrow/types.pxi                       |  40 +----
 16 files changed, 747 insertions(+), 182 deletions(-)

diff --git a/ci/scripts/integration_spark.sh b/ci/scripts/integration_spark.sh
index 9828a28..a45ed7a 100755
--- a/ci/scripts/integration_spark.sh
+++ b/ci/scripts/integration_spark.sh
@@ -22,6 +22,9 @@ source_dir=${1}
 spark_dir=${2}
 spark_version=${SPARK_VERSION:-master}
 
+# Use old behavior that always dropped tiemzones.
+export PYARROW_IGNORE_TIMEZONE=1
+
 if [ "${SPARK_VERSION:0:2}" == "2." ]; then
   # https://github.com/apache/spark/blob/master/docs/sql-pyspark-pandas-with-arrow.md#compatibility-setting-for-pyarrow--0150-and-spark-23x-24x
   export ARROW_PRE_0_15_IPC_FORMAT=1
diff --git a/cpp/src/arrow/compute/kernels/scalar_string.cc b/cpp/src/arrow/compute/kernels/scalar_string.cc
index 7e61617..0332be9 100644
--- a/cpp/src/arrow/compute/kernels/scalar_string.cc
+++ b/cpp/src/arrow/compute/kernels/scalar_string.cc
@@ -861,10 +861,10 @@ void AddBinaryLength(FunctionRegistry* registry) {
       applicator::ScalarUnaryNotNull<Int32Type, StringType, BinaryLength>::Exec;
   ArrayKernelExec exec_offset_64 =
       applicator::ScalarUnaryNotNull<Int64Type, LargeStringType, BinaryLength>::Exec;
-  for (const auto& input_type : {binary(), utf8()}) {
+  for (const auto input_type : {binary(), utf8()}) {
     DCHECK_OK(func->AddKernel({input_type}, int32(), exec_offset_32));
   }
-  for (const auto& input_type : {large_binary(), large_utf8()}) {
+  for (const auto input_type : {large_binary(), large_utf8()}) {
     DCHECK_OK(func->AddKernel({input_type}, int64(), exec_offset_64));
   }
   DCHECK_OK(registry->AddFunction(std::move(func)));
diff --git a/cpp/src/arrow/python/arrow_to_pandas.cc b/cpp/src/arrow/python/arrow_to_pandas.cc
index bc4e25b..47b62a3 100644
--- a/cpp/src/arrow/python/arrow_to_pandas.cc
+++ b/cpp/src/arrow/python/arrow_to_pandas.cc
@@ -17,9 +17,8 @@
 
 // Functions for pandas conversion via NumPy
 
-#include "arrow/python/numpy_interop.h"  // IWYU pragma: expand
-
 #include "arrow/python/arrow_to_pandas.h"
+#include "arrow/python/numpy_interop.h"  // IWYU pragma: expand
 
 #include <cmath>
 #include <cstdint>
@@ -642,15 +641,15 @@ inline Status ConvertStruct(const PandasOptions& options, const ChunkedArray& da
   std::vector<OwnedRef> fields_data(num_fields);
   OwnedRef dict_item;
 
-  // XXX(wesm): In ARROW-7723, we found as a result of ARROW-3789 that second
+  // In ARROW-7723, we found as a result of ARROW-3789 that second
   // through microsecond resolution tz-aware timestamps were being promoted to
   // use the DATETIME_NANO_TZ conversion path, yielding a datetime64[ns] NumPy
   // array in this function. PyArray_GETITEM returns datetime.datetime for
   // units second through microsecond but PyLong for nanosecond (because
-  // datetime.datetime does not support nanoseconds). We inserted this hack to
-  // preserve the <= 0.15.1 behavior until a better solution can be devised
+  // datetime.datetime does not support nanoseconds).
+  // We force the object conversion to preserve the value of the timezone.
+  // Nanoseconds are returned integers inside of structs.
   PandasOptions modified_options = options;
-  modified_options.ignore_timezone = true;
   modified_options.coerce_temporal_nanoseconds = false;
 
   for (int c = 0; c < data.num_chunks(); c++) {
@@ -658,8 +657,12 @@ inline Status ConvertStruct(const PandasOptions& options, const ChunkedArray& da
     // Convert the struct arrays first
     for (int32_t i = 0; i < num_fields; i++) {
       PyObject* numpy_array;
-      RETURN_NOT_OK(ConvertArrayToPandas(
-          modified_options, arr->field(static_cast<int>(i)), nullptr, &numpy_array));
+      std::shared_ptr<Array> field = arr->field(static_cast<int>(i));
+      // See notes above about timestamp conversion.  Don't blindly convert because
+      // timestamps in lists are handled differently.
+      modified_options.timestamp_as_object =
+          field->type()->id() == Type::TIMESTAMP ? true : options.timestamp_as_object;
+      RETURN_NOT_OK(ConvertArrayToPandas(modified_options, field, nullptr, &numpy_array));
       fields_data[i].reset(numpy_array);
     }
 
@@ -951,12 +954,39 @@ struct ObjectWriterVisitor {
   template <typename Type>
   enable_if_timestamp<Type, Status> Visit(const Type& type) {
     const TimeUnit::type unit = type.unit();
-    auto WrapValue = [unit](typename Type::c_type value, PyObject** out) {
+    OwnedRef tzinfo;
+
+    auto ConvertTimezoneNaive = [&](typename Type::c_type value, PyObject** out) {
       RETURN_NOT_OK(internal::PyDateTime_from_int(value, unit, out));
       RETURN_IF_PYERROR();
       return Status::OK();
     };
-    return ConvertAsPyObjects<Type>(options, data, WrapValue, out_values);
+    auto ConvertTimezoneAware = [&](typename Type::c_type value, PyObject** out) {
+      PyObject* naive_datetime;
+      RETURN_NOT_OK(ConvertTimezoneNaive(value, &naive_datetime));
+      // convert the timezone naive datetime object to timezone aware
+      *out = PyObject_CallMethod(tzinfo.obj(), "fromutc", "O", naive_datetime);
+      // the timezone naive object is no longer required
+      Py_DECREF(naive_datetime);
+      RETURN_IF_PYERROR();
+      return Status::OK();
+    };
+
+    if (!type.timezone().empty() && !options.ignore_timezone) {
+      // convert timezone aware
+      PyObject* tzobj;
+      ARROW_ASSIGN_OR_RAISE(tzobj, internal::StringToTzinfo(type.timezone()));
+      tzinfo.reset(tzobj);
+      RETURN_IF_PYERROR();
+      RETURN_NOT_OK(
+          ConvertAsPyObjects<Type>(options, data, ConvertTimezoneAware, out_values));
+    } else {
+      // convert timezone naive
+      RETURN_NOT_OK(
+          ConvertAsPyObjects<Type>(options, data, ConvertTimezoneNaive, out_values));
+    }
+
+    return Status::OK();
   }
 
   Status Visit(const Decimal128Type& type) {
@@ -1727,8 +1757,7 @@ static Status GetPandasWriterType(const ChunkedArray& data, const PandasOptions&
         // Nanoseconds are never out of bounds for pandas, so in that case
         // we don't convert to object
         *output_type = PandasWriter::OBJECT;
-      } else if (ts_type.timezone() != "" && !options.ignore_timezone) {
-        // XXX: ignore_timezone: hack here for ARROW-7723
+      } else if (!ts_type.timezone().empty()) {
         *output_type = PandasWriter::DATETIME_NANO_TZ;
       } else if (options.coerce_temporal_nanoseconds) {
         *output_type = PandasWriter::DATETIME_NANO;
diff --git a/cpp/src/arrow/python/arrow_to_pandas.h b/cpp/src/arrow/python/arrow_to_pandas.h
index 79a72bc..abf4bbd 100644
--- a/cpp/src/arrow/python/arrow_to_pandas.h
+++ b/cpp/src/arrow/python/arrow_to_pandas.h
@@ -56,8 +56,9 @@ struct PandasOptions {
   /// Coerce all date and timestamp to datetime64[ns]
   bool coerce_temporal_nanoseconds = false;
 
-  /// XXX(wesm): Hack for ARROW-7723 to opt out of DATETIME_NANO_TZ conversion
-  /// path
+  /// Used to maintain backwards compatibility for
+  /// timezone bugs (see ARROW-9528).  Should be removed
+  /// after Arrow 2.0 release.
   bool ignore_timezone = false;
 
   /// \brief If true, do not create duplicate PyObject versions of equal
diff --git a/cpp/src/arrow/python/datetime.cc b/cpp/src/arrow/python/datetime.cc
index 8cec87b..4eeab7f 100644
--- a/cpp/src/arrow/python/datetime.cc
+++ b/cpp/src/arrow/python/datetime.cc
@@ -14,22 +14,66 @@
 // KIND, either express or implied.  See the License for the
 // specific language governing permissions and limitations
 // under the License.
+#include "arrow/python/datetime.h"
 
 #include <algorithm>
 #include <chrono>
+#include <iomanip>
 #include <iostream>
 
 #include "arrow/python/common.h"
-#include "arrow/python/datetime.h"
+#include "arrow/python/helpers.h"
 #include "arrow/python/platform.h"
 #include "arrow/status.h"
 #include "arrow/type.h"
 #include "arrow/util/logging.h"
+#include "arrow/util/value_parsing.h"
 
 namespace arrow {
 namespace py {
 namespace internal {
 
+namespace {
+
+// Same as Regex '([+-])(0[0-9]|1[0-9]|2[0-3]):([0-5][0-9])$'.
+// GCC 4.9 doesn't support regex, so handcode until support for it
+// is dropped.
+bool MatchFixedOffset(const std::string& tz, util::string_view* sign,
+                      util::string_view* hour, util::string_view* minute) {
+  if (tz.size() < 5) {
+    return false;
+  }
+  const char* iter = tz.data();
+  if (*iter == '+' || *iter == '-') {
+    *sign = util::string_view(iter, 1);
+    iter++;
+    if (tz.size() < 6) {
+      return false;
+    }
+  }
+  if ((((*iter == '0' || *iter == '1') && *(iter + 1) >= '0' && *(iter + 1) <= '9') ||
+       (*iter == '2' && *(iter + 1) >= '0' && *(iter + 1) <= '3'))) {
+    *hour = util::string_view(iter, 2);
+    iter += 2;
+  } else {
+    return false;
+  }
+  if (*iter != ':') {
+    return false;
+  }
+  iter++;
+
+  if (*iter >= '0' && *iter <= '5' && *(iter + 1) >= '0' && *(iter + 1) <= '9') {
+    *minute = util::string_view(iter, 2);
+    iter += 2;
+  } else {
+    return false;
+  }
+  return iter == (tz.data() + tz.size());
+}
+
+}  // namespace
+
 PyDateTime_CAPI* datetime_api = nullptr;
 
 void InitDatetime() {
@@ -262,6 +306,132 @@ int64_t PyDate_to_days(PyDateTime_Date* pydate) {
                             PyDateTime_GET_DAY(pydate));
 }
 
+Result<int64_t> PyDateTime_utcoffset_s(PyObject* obj) {
+  // calculate offset from UTC timezone in seconds
+  // supports only PyDateTime_DateTime and PyDateTime_Time objects
+  OwnedRef pyoffset(PyObject_CallMethod(obj, "utcoffset", NULL));
+  RETURN_IF_PYERROR();
+  if (pyoffset.obj() != nullptr && pyoffset.obj() != Py_None) {
+    auto delta = reinterpret_cast<PyDateTime_Delta*>(pyoffset.obj());
+    return internal::PyDelta_to_s(delta);
+  } else {
+    return 0;
+  }
+}
+
+Result<std::string> PyTZInfo_utcoffset_hhmm(PyObject* pytzinfo) {
+  // attempt to convert timezone offset objects to "+/-{hh}:{mm}" format
+  OwnedRef pydelta_object(PyObject_CallMethod(pytzinfo, "utcoffset", "O", Py_None));
+  RETURN_IF_PYERROR();
+
+  if (!PyDelta_Check(pydelta_object.obj())) {
+    return Status::Invalid(
+        "Object returned by tzinfo.utcoffset(None) is not an instance of "
+        "datetime.timedelta");
+  }
+  auto pydelta = reinterpret_cast<PyDateTime_Delta*>(pydelta_object.obj());
+
+  // retrieve the offset as seconds
+  auto total_seconds = internal::PyDelta_to_s(pydelta);
+
+  // determine whether the offset is positive or negative
+  auto sign = (total_seconds < 0) ? "-" : "+";
+  total_seconds = abs(total_seconds);
+
+  // calculate offset components
+  int64_t hours, minutes, seconds;
+  seconds = split_time(total_seconds, 60, &minutes);
+  minutes = split_time(minutes, 60, &hours);
+  if (seconds > 0) {
+    // check there are no remaining seconds
+    return Status::Invalid("Offset must represent whole number of minutes");
+  }
+
+  // construct the timezone string
+  std::stringstream stream;
+  stream << sign << std::setfill('0') << std::setw(2) << hours << ":" << std::setfill('0')
+         << std::setw(2) << minutes;
+  return stream.str();
+}
+
+// Converted from python.  See https://github.com/apache/arrow/pull/7604
+// for details.
+Result<PyObject*> StringToTzinfo(const std::string& tz) {
+  util::string_view sign_str, hour_str, minute_str;
+  OwnedRef pytz;
+  RETURN_NOT_OK(internal::ImportModule("pytz", &pytz));
+
+  if (MatchFixedOffset(tz, &sign_str, &hour_str, &minute_str)) {
+    int sign = -1;
+    if (sign_str == "+") {
+      sign = 1;
+    }
+    OwnedRef fixed_offset;
+    RETURN_NOT_OK(internal::ImportFromModule(pytz.obj(), "FixedOffset", &fixed_offset));
+    uint32_t minutes, hours;
+    if (!::arrow::internal::ParseUnsigned(hour_str.data(), hour_str.size(), &hours) ||
+        !::arrow::internal::ParseUnsigned(minute_str.data(), minute_str.size(),
+                                          &minutes)) {
+      return Status::Invalid("Invalid timezone: ", tz);
+    }
+    OwnedRef total_minutes(PyLong_FromLong(
+        sign * ((static_cast<int>(hours) * 60) + static_cast<int>(minutes))));
+    RETURN_IF_PYERROR();
+    auto tzinfo =
+        PyObject_CallFunctionObjArgs(fixed_offset.obj(), total_minutes.obj(), NULL);
+    RETURN_IF_PYERROR();
+    return tzinfo;
+  }
+
+  OwnedRef timezone;
+  RETURN_NOT_OK(internal::ImportFromModule(pytz.obj(), "timezone", &timezone));
+  OwnedRef py_tz_string(
+      PyUnicode_FromStringAndSize(tz.c_str(), static_cast<Py_ssize_t>(tz.size())));
+  auto tzinfo = PyObject_CallFunctionObjArgs(timezone.obj(), py_tz_string.obj(), NULL);
+  RETURN_IF_PYERROR();
+  return tzinfo;
+}
+
+Result<std::string> TzinfoToString(PyObject* tzinfo) {
+  OwnedRef module_pytz;        // import pytz
+  OwnedRef module_datetime;    // import datetime
+  OwnedRef class_timezone;     // from datetime import timezone
+  OwnedRef class_fixedoffset;  // from pytz import _FixedOffset
+
+  // import necessary modules
+  RETURN_NOT_OK(internal::ImportModule("pytz", &module_pytz));
+  RETURN_NOT_OK(internal::ImportModule("datetime", &module_datetime));
+  // import necessary classes
+  RETURN_NOT_OK(
+      internal::ImportFromModule(module_pytz.obj(), "_FixedOffset", &class_fixedoffset));
+  RETURN_NOT_OK(
+      internal::ImportFromModule(module_datetime.obj(), "timezone", &class_timezone));
+
+  // check that it's a valid tzinfo object
+  if (!PyTZInfo_Check(tzinfo)) {
+    return Status::TypeError("Not an instance of datetime.tzinfo");
+  }
+
+  // if tzinfo is an instance of pytz._FixedOffset or datetime.timezone return the
+  // HH:MM offset string representation
+  if (PyObject_IsInstance(tzinfo, class_timezone.obj()) ||
+      PyObject_IsInstance(tzinfo, class_fixedoffset.obj())) {
+    return PyTZInfo_utcoffset_hhmm(tzinfo);
+  }
+
+  // attempt to call tzinfo.tzname(None)
+  OwnedRef tzname_object(PyObject_CallMethod(tzinfo, "tzname", "O", Py_None));
+  RETURN_IF_PYERROR();
+  if (PyUnicode_Check(tzname_object.obj())) {
+    std::string result;
+    RETURN_NOT_OK(internal::PyUnicode_AsStdString(tzname_object.obj(), &result));
+    return result;
+  }
+
+  // fall back to HH:MM offset string representation based on tzinfo.utcoffset(None)
+  return PyTZInfo_utcoffset_hhmm(tzinfo);
+}
+
 }  // namespace internal
 }  // namespace py
 }  // namespace arrow
diff --git a/cpp/src/arrow/python/datetime.h b/cpp/src/arrow/python/datetime.h
index a8b22da..4f3adb4 100644
--- a/cpp/src/arrow/python/datetime.h
+++ b/cpp/src/arrow/python/datetime.h
@@ -157,6 +157,32 @@ inline int64_t PyDelta_to_ns(PyDateTime_Delta* pytimedelta) {
   return PyDelta_to_us(pytimedelta) * 1000;
 }
 
+ARROW_PYTHON_EXPORT
+Result<int64_t> PyDateTime_utcoffset_s(PyObject* pydatetime);
+
+/// \brief Convert a time zone name into a time zone object.
+///
+/// Supported input strings are:
+/// * As used in the Olson time zone database (the "tz database" or
+///   "tzdata"), such as "America/New_York"
+/// * An absolute time zone offset of the form +XX:XX or -XX:XX, such as +07:30
+/// GIL must be held when calling this method.
+ARROW_PYTHON_EXPORT
+Result<PyObject*> StringToTzinfo(const std::string& tz);
+
+/// \brief Convert a time zone object to a string representation.
+///
+/// The output strings are:
+/// * An absolute time zone offset of the form +XX:XX or -XX:XX, such as +07:30
+///   if the input object is either an instance of pytz._FixedOffset or
+///   datetime.timedelta
+/// * The timezone's name if the input object's tzname() method returns with a
+///   non-empty timezone name such as "UTC" or "America/New_York"
+///
+/// GIL must be held when calling this method.
+ARROW_PYTHON_EXPORT
+Result<std::string> TzinfoToString(PyObject* pytzinfo);
+
 }  // namespace internal
 }  // namespace py
 }  // namespace arrow
diff --git a/cpp/src/arrow/python/inference.cc b/cpp/src/arrow/python/inference.cc
index c2fc06e..d1ce2c2 100644
--- a/cpp/src/arrow/python/inference.cc
+++ b/cpp/src/arrow/python/inference.cc
@@ -295,10 +295,7 @@ class TypeInferrer {
         int_count_(0),
         date_count_(0),
         time_count_(0),
-        timestamp_second_count_(0),
-        timestamp_milli_count_(0),
         timestamp_micro_count_(0),
-        timestamp_nano_count_(0),
         duration_count_(0),
         float_count_(0),
         binary_count_(0),
@@ -331,6 +328,13 @@ class TypeInferrer {
     } else if (internal::IsPyInteger(obj)) {
       ++int_count_;
     } else if (PyDateTime_Check(obj)) {
+      // infer timezone from the first encountered datetime object
+      if (!timestamp_micro_count_) {
+        OwnedRef tzinfo(PyObject_GetAttrString(obj, "tzinfo"));
+        if (tzinfo.obj() != nullptr && tzinfo.obj() != Py_None) {
+          ARROW_ASSIGN_OR_RAISE(timezone_, internal::TzinfoToString(tzinfo.obj()));
+        }
+      }
       ++timestamp_micro_count_;
       *keep_going = make_unions_;
     } else if (PyDelta_Check(obj)) {
@@ -458,14 +462,8 @@ class TypeInferrer {
       *out = date32();
     } else if (time_count_) {
       *out = time64(TimeUnit::MICRO);
-    } else if (timestamp_nano_count_) {
-      *out = timestamp(TimeUnit::NANO);
     } else if (timestamp_micro_count_) {
-      *out = timestamp(TimeUnit::MICRO);
-    } else if (timestamp_milli_count_) {
-      *out = timestamp(TimeUnit::MILLI);
-    } else if (timestamp_second_count_) {
-      *out = timestamp(TimeUnit::SECOND);
+      *out = timestamp(TimeUnit::MICRO, timezone_);
     } else if (duration_count_) {
       *out = duration(TimeUnit::MICRO);
     } else if (bool_count_) {
@@ -597,10 +595,8 @@ class TypeInferrer {
   int64_t int_count_;
   int64_t date_count_;
   int64_t time_count_;
-  int64_t timestamp_second_count_;
-  int64_t timestamp_milli_count_;
   int64_t timestamp_micro_count_;
-  int64_t timestamp_nano_count_;
+  std::string timezone_;
   int64_t duration_count_;
   int64_t float_count_;
   int64_t binary_count_;
diff --git a/cpp/src/arrow/python/python_to_arrow.cc b/cpp/src/arrow/python/python_to_arrow.cc
index d75c0ac..098f298 100644
--- a/cpp/src/arrow/python/python_to_arrow.cc
+++ b/cpp/src/arrow/python/python_to_arrow.cc
@@ -193,10 +193,11 @@ struct ValueConverter<Date64Type> {
 
 template <>
 struct ValueConverter<Time32Type> {
-  static inline Result<int32_t> FromPython(PyObject* obj, TimeUnit::type unit) {
+  static inline Result<int32_t> FromPython(PyObject* obj, TimeUnit::type unit,
+                                           bool /*ignore_timezone*/) {
     int32_t value;
     if (PyTime_Check(obj)) {
-      // datetime.time stores microsecond resolution
+      // TODO(kszucs): consider to raise if a timezone aware time object is encountered
       switch (unit) {
         case TimeUnit::SECOND:
           value = static_cast<int32_t>(internal::PyTime_to_s(obj));
@@ -208,6 +209,7 @@ struct ValueConverter<Time32Type> {
           return Status::UnknownError("Invalid time unit");
       }
     } else {
+      // TODO(kszucs): validate maximum value?
       RETURN_NOT_OK(internal::CIntFromPython(obj, &value, "Integer too large for int32"));
     }
     return value;
@@ -216,10 +218,11 @@ struct ValueConverter<Time32Type> {
 
 template <>
 struct ValueConverter<Time64Type> {
-  static inline Result<int64_t> FromPython(PyObject* obj, TimeUnit::type unit) {
+  static inline Result<int64_t> FromPython(PyObject* obj, TimeUnit::type unit,
+                                           bool /*ignore_timezone=*/) {
     int64_t value;
     if (PyTime_Check(obj)) {
-      // datetime.time stores microsecond resolution
+      // TODO(kszucs): consider to raise if a timezone aware time object is encountered
       switch (unit) {
         case TimeUnit::MICRO:
           value = internal::PyTime_to_us(obj);
@@ -231,6 +234,7 @@ struct ValueConverter<Time64Type> {
           return Status::UnknownError("Invalid time unit");
       }
     } else {
+      // TODO(kszucs): validate maximum value?
       RETURN_NOT_OK(internal::CIntFromPython(obj, &value, "Integer too large for int64"));
     }
     return value;
@@ -239,22 +243,27 @@ struct ValueConverter<Time64Type> {
 
 template <>
 struct ValueConverter<TimestampType> {
-  static inline Result<int64_t> FromPython(PyObject* obj, TimeUnit::type unit) {
+  static inline Result<int64_t> FromPython(PyObject* obj, TimeUnit::type unit,
+                                           bool ignore_timezone) {
     int64_t value;
     if (PyDateTime_Check(obj)) {
+      ARROW_ASSIGN_OR_RAISE(int64_t offset, internal::PyDateTime_utcoffset_s(obj));
+      if (ignore_timezone) {
+        offset = 0;
+      }
       auto dt = reinterpret_cast<PyDateTime_DateTime*>(obj);
       switch (unit) {
         case TimeUnit::SECOND:
-          value = internal::PyDateTime_to_s(dt);
+          value = internal::PyDateTime_to_s(dt) - offset;
           break;
         case TimeUnit::MILLI:
-          value = internal::PyDateTime_to_ms(dt);
+          value = internal::PyDateTime_to_ms(dt) - offset * 1000;
           break;
         case TimeUnit::MICRO:
-          value = internal::PyDateTime_to_us(dt);
+          value = internal::PyDateTime_to_us(dt) - offset * 1000 * 1000;
           break;
         case TimeUnit::NANO:
-          value = internal::PyDateTime_to_ns(dt);
+          value = internal::PyDateTime_to_ns(dt) - offset * 1000 * 1000 * 1000;
           break;
         default:
           return Status::UnknownError("Invalid time unit");
@@ -285,7 +294,8 @@ struct ValueConverter<TimestampType> {
 
 template <>
 struct ValueConverter<DurationType> {
-  static inline Result<int64_t> FromPython(PyObject* obj, TimeUnit::type unit) {
+  static inline Result<int64_t> FromPython(PyObject* obj, TimeUnit::type unit,
+                                           bool /*ignore_timezone*/) {
     int64_t value;
     if (PyDelta_Check(obj)) {
       auto dt = reinterpret_cast<PyDateTime_Delta*>(obj);
@@ -391,7 +401,8 @@ class SeqConverter;
 
 // Forward-declare converter factory
 Status GetConverter(const std::shared_ptr<DataType>& type, bool from_pandas,
-                    bool strict_conversions, std::unique_ptr<SeqConverter>* out);
+                    bool strict_conversions, bool ignore_timezone,
+                    std::unique_ptr<SeqConverter>* out);
 
 // Marshal Python sequence (list, tuple, etc.) to Arrow array
 class SeqConverter {
@@ -526,16 +537,19 @@ class PrimitiveConverter : public TypedConverter<Type, null_coding> {
 template <typename Type, NullCoding null_coding>
 class TimeConverter : public TypedConverter<Type, null_coding> {
  public:
-  explicit TimeConverter(TimeUnit::type unit) : unit_(unit) {}
+  explicit TimeConverter(TimeUnit::type unit, bool ignore_timezone)
+      : unit_(unit), ignore_timezone_(ignore_timezone) {}
 
   // TODO(kszucs): support numpy values for date and time converters
   Status AppendValue(PyObject* obj) override {
-    ARROW_ASSIGN_OR_RAISE(auto value, ValueConverter<Type>::FromPython(obj, unit_));
+    ARROW_ASSIGN_OR_RAISE(auto value,
+                          ValueConverter<Type>::FromPython(obj, unit_, ignore_timezone_));
     return this->typed_builder_->Append(value);
   }
 
  protected:
   TimeUnit::type unit_;
+  bool ignore_timezone_;
 };
 
 // TODO(kszucs): move it to the type_traits
@@ -571,8 +585,10 @@ class TemporalConverter : public TimeConverter<Type, null_coding> {
         return this->typed_builder_->AppendNull();
       }
     } else {
-      // convert builtin python objects
-      ARROW_ASSIGN_OR_RAISE(value, ValueConverter<Type>::FromPython(obj, this->unit_));
+      ARROW_ASSIGN_OR_RAISE(
+          value,
+          ValueConverter<Type>::FromPython(
+              obj, this->unit_, TimeConverter<Type, null_coding>::ignore_timezone_));
     }
     return this->typed_builder_->Append(value);
   }
@@ -713,16 +729,19 @@ class BaseListConverter : public TypedConverter<TypeClass, null_coding> {
  public:
   using BuilderType = typename TypeTraits<TypeClass>::BuilderType;
 
-  explicit BaseListConverter(bool from_pandas, bool strict_conversions)
-      : from_pandas_(from_pandas), strict_conversions_(strict_conversions) {}
+  explicit BaseListConverter(bool from_pandas, bool strict_conversions,
+                             bool ignore_timezone)
+      : from_pandas_(from_pandas),
+        strict_conversions_(strict_conversions),
+        ignore_timezone_(ignore_timezone) {}
 
   Status Init(ArrayBuilder* builder) override {
     this->builder_ = builder;
     this->typed_builder_ = checked_cast<BuilderType*>(builder);
 
     this->value_type_ = checked_cast<const TypeClass&>(*builder->type()).value_type();
-    RETURN_NOT_OK(
-        GetConverter(value_type_, from_pandas_, strict_conversions_, &value_converter_));
+    RETURN_NOT_OK(GetConverter(value_type_, from_pandas_, strict_conversions_,
+                               ignore_timezone_, &value_converter_));
     return this->value_converter_->Init(this->typed_builder_->value_builder());
   }
 
@@ -832,8 +851,9 @@ class BaseListConverter : public TypedConverter<TypeClass, null_coding> {
  protected:
   std::shared_ptr<DataType> value_type_;
   std::unique_ptr<SeqConverter> value_converter_;
-  bool from_pandas_;
-  bool strict_conversions_;
+  const bool from_pandas_;
+  const bool strict_conversions_;
+  const bool ignore_timezone_;
 };
 
 template <typename TypeClass, NullCoding null_coding>
@@ -893,8 +913,8 @@ class MapConverter : public BaseListConverter<MapType, null_coding> {
  public:
   using BASE = BaseListConverter<MapType, null_coding>;
 
-  explicit MapConverter(bool from_pandas, bool strict_conversions)
-      : BASE(from_pandas, strict_conversions), key_builder_(nullptr) {}
+  explicit MapConverter(bool from_pandas, bool strict_conversions, bool ignore_timezone)
+      : BASE(from_pandas, strict_conversions, ignore_timezone), key_builder_(nullptr) {}
 
   Status Append(PyObject* obj) override {
     RETURN_NOT_OK(BASE::Append(obj));
@@ -936,8 +956,11 @@ class MapConverter : public BaseListConverter<MapType, null_coding> {
 template <NullCoding null_coding>
 class StructConverter : public TypedConverter<StructType, null_coding> {
  public:
-  explicit StructConverter(bool from_pandas, bool strict_conversions)
-      : from_pandas_(from_pandas), strict_conversions_(strict_conversions) {}
+  explicit StructConverter(bool from_pandas, bool strict_conversions,
+                           bool ignore_timezone)
+      : from_pandas_(from_pandas),
+        strict_conversions_(strict_conversions),
+        ignore_timezone_(ignore_timezone) {}
 
   Status Init(ArrayBuilder* builder) override {
     this->builder_ = builder;
@@ -957,8 +980,8 @@ class StructConverter : public TypedConverter<StructType, null_coding> {
       std::shared_ptr<DataType> field_type(struct_type->field(i)->type());
 
       std::unique_ptr<SeqConverter> value_converter;
-      RETURN_NOT_OK(
-          GetConverter(field_type, from_pandas_, strict_conversions_, &value_converter));
+      RETURN_NOT_OK(GetConverter(field_type, from_pandas_, strict_conversions_,
+                                 ignore_timezone_, &value_converter));
       RETURN_NOT_OK(value_converter->Init(this->typed_builder_->field_builder(i)));
       value_converters_.push_back(std::move(value_converter));
 
@@ -1076,6 +1099,7 @@ class StructConverter : public TypedConverter<StructType, null_coding> {
   } dict_key_kind_ = DictKeyKind::UNKNOWN;
   bool from_pandas_;
   bool strict_conversions_;
+  bool ignore_timezone_;
 };
 
 template <NullCoding null_coding>
@@ -1112,7 +1136,7 @@ class DecimalConverter : public TypedConverter<arrow::Decimal128Type, null_codin
 // Dynamic constructor for sequence converters
 template <NullCoding null_coding>
 Status GetConverterFlat(const std::shared_ptr<DataType>& type, bool strict_conversions,
-                        std::unique_ptr<SeqConverter>* out) {
+                        bool ignore_timezone, std::unique_ptr<SeqConverter>* out) {
   switch (type->id()) {
     SIMPLE_CONVERTER_CASE(NA, NullConverter);
     PRIMITIVE(BOOL, BooleanType);
@@ -1161,25 +1185,28 @@ Status GetConverterFlat(const std::shared_ptr<DataType>& type, bool strict_conve
       }
       break;
     case Type::TIME32: {
-      *out = std::unique_ptr<SeqConverter>(new TimeConverter<Time32Type, null_coding>(
-          checked_cast<const Time32Type&>(*type).unit()));
+      auto unit = checked_cast<const Time32Type&>(*type).unit();
+      *out = std::unique_ptr<SeqConverter>(
+          new TimeConverter<Time32Type, null_coding>(unit, ignore_timezone));
       break;
     }
     case Type::TIME64: {
-      *out = std::unique_ptr<SeqConverter>(new TimeConverter<Time64Type, null_coding>(
-          checked_cast<const Time64Type&>(*type).unit()));
+      auto unit = checked_cast<const Time64Type&>(*type).unit();
+      *out = std::unique_ptr<SeqConverter>(
+          new TimeConverter<Time64Type, null_coding>(unit, ignore_timezone));
       break;
     }
     case Type::TIMESTAMP: {
-      *out =
-          std::unique_ptr<SeqConverter>(new TemporalConverter<TimestampType, null_coding>(
-              checked_cast<const TimestampType&>(*type).unit()));
+      auto unit = checked_cast<const TimestampType&>(*type).unit();
+      *out = std::unique_ptr<SeqConverter>(
+          new TemporalConverter<TimestampType, null_coding>(unit, ignore_timezone));
       break;
     }
     case Type::DURATION: {
+      auto unit = checked_cast<const DurationType&>(*type).unit();
       *out =
           std::unique_ptr<SeqConverter>(new TemporalConverter<DurationType, null_coding>(
-              checked_cast<const DurationType&>(*type).unit()));
+              unit, /*ignore_timezone=*/false));
       break;
     }
     default:
@@ -1190,7 +1217,8 @@ Status GetConverterFlat(const std::shared_ptr<DataType>& type, bool strict_conve
 }
 
 Status GetConverter(const std::shared_ptr<DataType>& type, bool from_pandas,
-                    bool strict_conversions, std::unique_ptr<SeqConverter>* out) {
+                    bool strict_conversions, bool ignore_timezone,
+                    std::unique_ptr<SeqConverter>* out) {
   if (from_pandas) {
     // ARROW-842: If pandas is not installed then null checks will be less
     // comprehensive, but that is okay.
@@ -1202,53 +1230,53 @@ Status GetConverter(const std::shared_ptr<DataType>& type, bool from_pandas,
       if (from_pandas) {
         *out = std::unique_ptr<SeqConverter>(
             new ListConverter<ListType, NullCoding::PANDAS_SENTINELS>(
-                from_pandas, strict_conversions));
+                from_pandas, strict_conversions, ignore_timezone));
       } else {
         *out = std::unique_ptr<SeqConverter>(
-            new ListConverter<ListType, NullCoding::NONE_ONLY>(from_pandas,
-                                                               strict_conversions));
+            new ListConverter<ListType, NullCoding::NONE_ONLY>(
+                from_pandas, strict_conversions, ignore_timezone));
       }
       return Status::OK();
     case Type::LARGE_LIST:
       if (from_pandas) {
         *out = std::unique_ptr<SeqConverter>(
             new ListConverter<LargeListType, NullCoding::PANDAS_SENTINELS>(
-                from_pandas, strict_conversions));
+                from_pandas, strict_conversions, ignore_timezone));
       } else {
         *out = std::unique_ptr<SeqConverter>(
-            new ListConverter<LargeListType, NullCoding::NONE_ONLY>(from_pandas,
-                                                                    strict_conversions));
+            new ListConverter<LargeListType, NullCoding::NONE_ONLY>(
+                from_pandas, strict_conversions, ignore_timezone));
       }
       return Status::OK();
     case Type::MAP:
       if (from_pandas) {
         *out =
             std::unique_ptr<SeqConverter>(new MapConverter<NullCoding::PANDAS_SENTINELS>(
-                from_pandas, strict_conversions));
+                from_pandas, strict_conversions, ignore_timezone));
       } else {
-        *out = std::unique_ptr<SeqConverter>(
-            new MapConverter<NullCoding::NONE_ONLY>(from_pandas, strict_conversions));
+        *out = std::unique_ptr<SeqConverter>(new MapConverter<NullCoding::NONE_ONLY>(
+            from_pandas, strict_conversions, ignore_timezone));
       }
       return Status::OK();
     case Type::FIXED_SIZE_LIST:
       if (from_pandas) {
         *out = std::unique_ptr<SeqConverter>(
-            new FixedSizeListConverter<NullCoding::PANDAS_SENTINELS>(from_pandas,
-                                                                     strict_conversions));
+            new FixedSizeListConverter<NullCoding::PANDAS_SENTINELS>(
+                from_pandas, strict_conversions, ignore_timezone));
       } else {
         *out = std::unique_ptr<SeqConverter>(
-            new FixedSizeListConverter<NullCoding::NONE_ONLY>(from_pandas,
-                                                              strict_conversions));
+            new FixedSizeListConverter<NullCoding::NONE_ONLY>(
+                from_pandas, strict_conversions, ignore_timezone));
       }
       return Status::OK();
     case Type::STRUCT:
       if (from_pandas) {
         *out = std::unique_ptr<SeqConverter>(
-            new StructConverter<NullCoding::PANDAS_SENTINELS>(from_pandas,
-                                                              strict_conversions));
+            new StructConverter<NullCoding::PANDAS_SENTINELS>(
+                from_pandas, strict_conversions, ignore_timezone));
       } else {
-        *out = std::unique_ptr<SeqConverter>(
-            new StructConverter<NullCoding::NONE_ONLY>(from_pandas, strict_conversions));
+        *out = std::unique_ptr<SeqConverter>(new StructConverter<NullCoding::NONE_ONLY>(
+            from_pandas, strict_conversions, ignore_timezone));
       }
       return Status::OK();
     default:
@@ -1256,10 +1284,11 @@ Status GetConverter(const std::shared_ptr<DataType>& type, bool from_pandas,
   }
 
   if (from_pandas) {
-    RETURN_NOT_OK(
-        GetConverterFlat<NullCoding::PANDAS_SENTINELS>(type, strict_conversions, out));
+    RETURN_NOT_OK(GetConverterFlat<NullCoding::PANDAS_SENTINELS>(type, strict_conversions,
+                                                                 ignore_timezone, out));
   } else {
-    RETURN_NOT_OK(GetConverterFlat<NullCoding::NONE_ONLY>(type, strict_conversions, out));
+    RETURN_NOT_OK(GetConverterFlat<NullCoding::NONE_ONLY>(type, strict_conversions,
+                                                          ignore_timezone, out));
   }
   return Status::OK();
 }
@@ -1330,6 +1359,10 @@ Status ConvertPySequence(PyObject* sequence_source, PyObject* mask,
 
   if (options.type == nullptr) {
     RETURN_NOT_OK(InferArrowType(seq, mask, options.from_pandas, &real_type));
+    if (options.ignore_timezone && real_type->id() == Type::TIMESTAMP) {
+      const auto& ts_type = checked_cast<const TimestampType&>(*real_type);
+      real_type = timestamp(ts_type.unit());
+    }
   } else {
     real_type = options.type;
     strict_conversions = true;
@@ -1338,8 +1371,8 @@ Status ConvertPySequence(PyObject* sequence_source, PyObject* mask,
 
   // Create the sequence converter, initialize with the builder
   std::unique_ptr<SeqConverter> converter;
-  RETURN_NOT_OK(
-      GetConverter(real_type, options.from_pandas, strict_conversions, &converter));
+  RETURN_NOT_OK(GetConverter(real_type, options.from_pandas, strict_conversions,
+                             options.ignore_timezone, &converter));
 
   // Create ArrayBuilder for type, then pass into the SeqConverter
   // instance. The reason this is created here rather than in GetConverter is
diff --git a/cpp/src/arrow/python/python_to_arrow.h b/cpp/src/arrow/python/python_to_arrow.h
index 5c8052a..5108e75 100644
--- a/cpp/src/arrow/python/python_to_arrow.h
+++ b/cpp/src/arrow/python/python_to_arrow.h
@@ -54,8 +54,12 @@ struct PyConversionOptions {
   // Memory pool to use for allocations
   MemoryPool* pool;
 
-  // Default false
-  bool from_pandas;
+  bool from_pandas = false;
+
+  /// Used to maintain backwards compatibility for
+  /// timezone bugs (see ARROW-9528).  Should be removed
+  /// after Arrow 2.0 release.
+  bool ignore_timezone = false;
 };
 
 /// \brief Convert sequence (list, generator, NumPy array with dtype object) of
diff --git a/python/pyarrow/array.pxi b/python/pyarrow/array.pxi
index ac26ecc..34417da 100644
--- a/python/pyarrow/array.pxi
+++ b/python/pyarrow/array.pxi
@@ -15,6 +15,7 @@
 # specific language governing permissions and limitations
 # under the License.
 
+import os
 import warnings
 
 
@@ -31,6 +32,7 @@ cdef _sequence_to_array(object sequence, object mask, object size,
 
     options.pool = pool
     options.from_pandas = from_pandas
+    options.ignore_timezone = os.environ.get('PYARROW_IGNORE_TIMEZONE', False)
 
     cdef shared_ptr[CChunkedArray] out
 
@@ -730,6 +732,7 @@ cdef PandasOptions _convert_pandas_options(dict options):
     result.safe_cast = options['safe']
     result.split_blocks = options['split_blocks']
     result.self_destruct = options['self_destruct']
+    result.ignore_timezone = os.environ.get('PYARROW_IGNORE_TIMEZONE', False)
     return result
 
 
@@ -1287,7 +1290,9 @@ cdef _array_like_to_pandas(obj, options):
     result = pandas_api.series(arr, dtype=dtype, name=name)
 
     if (isinstance(original_type, TimestampType) and
-            original_type.tz is not None):
+            original_type.tz is not None and
+            # can be object dtype for non-ns and timestamp_as_object=True
+            result.dtype.kind == "M"):
         from pyarrow.pandas_compat import make_tz_aware
         result = make_tz_aware(result, original_type.tz)
 
diff --git a/python/pyarrow/includes/libarrow.pxd b/python/pyarrow/includes/libarrow.pxd
index 555688a..f25e376 100644
--- a/python/pyarrow/includes/libarrow.pxd
+++ b/python/pyarrow/includes/libarrow.pxd
@@ -1753,6 +1753,7 @@ cdef extern from "arrow/python/api.h" namespace "arrow::py" nogil:
         int64_t size
         CMemoryPool* pool
         c_bool from_pandas
+        c_bool ignore_timezone
 
     # TODO Some functions below are not actually "nogil"
 
@@ -1875,6 +1876,7 @@ cdef extern from "arrow/python/api.h" namespace "arrow::py" nogil:
         c_bool timestamp_as_object
         c_bool use_threads
         c_bool coerce_temporal_nanoseconds
+        c_bool ignore_timezone
         c_bool deduplicate_objects
         c_bool safe_cast
         c_bool split_blocks
@@ -1927,6 +1929,9 @@ cdef extern from "arrow/python/api.h" namespace "arrow::py::internal" nogil:
     CTimePoint TimePoint_from_s(double val)
     CTimePoint TimePoint_from_ns(int64_t val)
 
+    CResult[c_string] TzinfoToString(PyObject* pytzinfo)
+    CResult[PyObject*] StringToTzinfo(c_string)
+
 
 cdef extern from 'arrow/python/init.h':
     int arrow_init_numpy() except -1
diff --git a/python/pyarrow/tests/test_array.py b/python/pyarrow/tests/test_array.py
index a1bfc96..7c1f024 100644
--- a/python/pyarrow/tests/test_array.py
+++ b/python/pyarrow/tests/test_array.py
@@ -32,6 +32,7 @@ try:
     import pickle5
 except ImportError:
     pickle5 = None
+import pytz
 
 import pyarrow as pa
 import pyarrow.tests.strategies as past
@@ -309,6 +310,8 @@ def test_nulls(ty):
 def test_array_from_scalar():
     today = datetime.date.today()
     now = datetime.datetime.now()
+    now_utc = now.replace(tzinfo=pytz.utc)
+    now_with_tz = now_utc.astimezone(pytz.timezone('US/Eastern'))
     oneday = datetime.timedelta(days=1)
 
     cases = [
@@ -326,6 +329,14 @@ def test_array_from_scalar():
         (pa.scalar(True), 11, pa.array([True] * 11)),
         (today, 2, pa.array([today] * 2)),
         (now, 10, pa.array([now] * 10)),
+        (
+            now_with_tz,
+            2,
+            pa.array(
+                [now_utc] * 2,
+                type=pa.timestamp('us', tz=pytz.timezone('US/Eastern'))
+            )
+        ),
         (now.time(), 9, pa.array([now.time()] * 9)),
         (oneday, 4, pa.array([oneday] * 4)),
         (False, 9, pa.array([False] * 9)),
@@ -341,8 +352,8 @@ def test_array_from_scalar():
     for value, size, expected in cases:
         arr = pa.repeat(value, size)
         assert len(arr) == size
+        assert arr.type.equals(expected.type)
         assert arr.equals(expected)
-
         if expected.type == pa.null():
             assert arr.null_count == size
         else:
@@ -1804,6 +1815,15 @@ def test_array_from_numpy_datetimeD():
     assert result.equals(expected)
 
 
+def test_array_from_naive_datetimes():
+    arr = pa.array([
+        None,
+        datetime.datetime(2017, 4, 4, 12, 11, 10),
+        datetime.datetime(2018, 1, 1, 0, 2, 0)
+    ])
+    assert arr.type == pa.timestamp('us', tz=None)
+
+
 @pytest.mark.parametrize(('dtype', 'type'), [
     ('datetime64[s]', pa.timestamp('s')),
     ('datetime64[ms]', pa.timestamp('ms')),
diff --git a/python/pyarrow/tests/test_convert_builtin.py b/python/pyarrow/tests/test_convert_builtin.py
index 2bdb640..d50d283 100644
--- a/python/pyarrow/tests/test_convert_builtin.py
+++ b/python/pyarrow/tests/test_convert_builtin.py
@@ -797,6 +797,70 @@ def test_date32_overflow():
         pa.array(data3, type=pa.date32())
 
 
+@pytest.mark.parametrize(('time_type', 'unit', 'int_type'), [
+    (pa.time32, 's', 'int32'),
+    (pa.time32, 'ms', 'int32'),
+    (pa.time64, 'us', 'int64'),
+    (pa.time64, 'ns', 'int64'),
+])
+def test_sequence_time_with_timezone(time_type, unit, int_type):
+    def expected_integer_value(t):
+        # only use with utc time object because it doesn't adjust with the
+        # offset
+        units = ['s', 'ms', 'us', 'ns']
+        multiplier = 10**(units.index(unit) * 3)
+        if t is None:
+            return None
+        seconds = (
+            t.hour * 3600 +
+            t.minute * 60 +
+            t.second +
+            t.microsecond * 10**-6
+        )
+        return int(seconds * multiplier)
+
+    def expected_time_value(t):
+        # only use with utc time object because it doesn't adjust with the
+        # time objects tzdata
+        if unit == 's':
+            return t.replace(microsecond=0)
+        elif unit == 'ms':
+            return t.replace(microsecond=(t.microsecond // 1000) * 1000)
+        else:
+            return t
+
+    # only timezone naive times are supported in arrow
+    data = [
+        datetime.time(8, 23, 34, 123456),
+        datetime.time(5, 0, 0, 1000),
+        None,
+        datetime.time(1, 11, 56, 432539),
+        datetime.time(23, 10, 0, 437699)
+    ]
+
+    ty = time_type(unit)
+    arr = pa.array(data, type=ty)
+    assert len(arr) == 5
+    assert arr.type == ty
+    assert arr.null_count == 1
+
+    # test that the underlying integers are UTC values
+    values = arr.cast(int_type)
+    expected = list(map(expected_integer_value, data))
+    assert values.to_pylist() == expected
+
+    # test that the scalars are datetime.time objects with UTC timezone
+    assert arr[0].as_py() == expected_time_value(data[0])
+    assert arr[1].as_py() == expected_time_value(data[1])
+    assert arr[2].as_py() is None
+    assert arr[3].as_py() == expected_time_value(data[3])
+    assert arr[4].as_py() == expected_time_value(data[4])
+
+    def tz(hours, minutes=0):
+        offset = datetime.timedelta(hours=hours, minutes=minutes)
+        return datetime.timezone(offset)
+
+
 def test_sequence_timestamp():
     data = [
         datetime.datetime(2007, 7, 13, 1, 23, 34, 123456),
@@ -817,6 +881,137 @@ def test_sequence_timestamp():
                                                46, 57, 437699)
 
 
+@pytest.mark.parametrize('timezone', [
+    None,
+    'UTC',
+    'Europe/Budapest',
+])
+@pytest.mark.parametrize('unit', [
+    's',
+    'ms',
+    'us',
+    'ns'
+])
+def test_sequence_timestamp_with_timezone(timezone, unit):
+    def expected_integer_value(dt):
+        units = ['s', 'ms', 'us', 'ns']
+        multiplier = 10**(units.index(unit) * 3)
+        if dt is None:
+            return None
+        else:
+            # avoid float precision issues
+            ts = decimal.Decimal(str(dt.timestamp()))
+            return int(ts * multiplier)
+
+    def expected_datetime_value(dt):
+        if dt is None:
+            return None
+
+        if unit == 's':
+            dt = dt.replace(microsecond=0)
+        elif unit == 'ms':
+            dt = dt.replace(microsecond=(dt.microsecond // 1000) * 1000)
+
+        # adjust the timezone
+        if timezone is None:
+            # make datetime timezone unaware
+            return dt.replace(tzinfo=None)
+        else:
+            # convert to the expected timezone
+            return dt.astimezone(pytz.timezone(timezone))
+
+    data = [
+        datetime.datetime(2007, 7, 13, 8, 23, 34, 123456),  # naive
+        pytz.utc.localize(
+            datetime.datetime(2008, 1, 5, 5, 0, 0, 1000)
+        ),
+        None,
+        pytz.timezone('US/Eastern').localize(
+            datetime.datetime(2006, 1, 13, 12, 34, 56, 432539)
+        ),
+        pytz.timezone('Europe/Moscow').localize(
+            datetime.datetime(2010, 8, 13, 5, 0, 0, 437699)
+        ),
+    ]
+    utcdata = [
+        pytz.utc.localize(data[0]),
+        data[1],
+        None,
+        data[3].astimezone(pytz.utc),
+        data[4].astimezone(pytz.utc),
+    ]
+
+    ty = pa.timestamp(unit, tz=timezone)
+    arr = pa.array(data, type=ty)
+    assert len(arr) == 5
+    assert arr.type == ty
+    assert arr.null_count == 1
+
+    # test that the underlying integers are UTC values
+    values = arr.cast('int64')
+    expected = list(map(expected_integer_value, utcdata))
+    assert values.to_pylist() == expected
+
+    # test that the scalars are datetimes with the correct timezone
+    for i in range(len(arr)):
+        assert arr[i].as_py() == expected_datetime_value(utcdata[i])
+
+
+def test_sequence_timestamp_with_timezone_inference():
+    data = [
+        datetime.datetime(2007, 7, 13, 8, 23, 34, 123456),  # naive
+        pytz.utc.localize(
+            datetime.datetime(2008, 1, 5, 5, 0, 0, 1000)
+        ),
+        None,
+        pytz.timezone('US/Eastern').localize(
+            datetime.datetime(2006, 1, 13, 12, 34, 56, 432539)
+        ),
+        pytz.timezone('Europe/Moscow').localize(
+            datetime.datetime(2010, 8, 13, 5, 0, 0, 437699)
+        ),
+    ]
+    expected = [
+        pa.timestamp('us', tz=None),
+        pa.timestamp('us', tz='UTC'),
+        pa.timestamp('us', tz=None),
+        pa.timestamp('us', tz='US/Eastern'),
+        pa.timestamp('us', tz='Europe/Moscow')
+    ]
+    for dt, expected_type in zip(data, expected):
+        prepended = [dt] + data
+        arr = pa.array(prepended)
+        assert arr.type == expected_type
+
+
+@pytest.mark.pandas
+def test_sequence_timestamp_from_mixed_builtin_and_pandas_datetimes():
+    import pandas as pd
+
+    data = [
+        pd.Timestamp(1184307814123456123, tz=pytz.timezone('US/Eastern'),
+                     unit='ns'),
+        datetime.datetime(2007, 7, 13, 8, 23, 34, 123456),  # naive
+        pytz.utc.localize(
+            datetime.datetime(2008, 1, 5, 5, 0, 0, 1000)
+        ),
+        None,
+    ]
+    utcdata = [
+        data[0].astimezone(pytz.utc),
+        pytz.utc.localize(data[1]),
+        data[2].astimezone(pytz.utc),
+        None,
+    ]
+
+    arr = pa.array(data)
+    assert arr.type == pa.timestamp('us', tz='US/Eastern')
+
+    values = arr.cast('int64')
+    expected = [int(dt.timestamp() * 10**6) if dt else None for dt in utcdata]
+    assert values.to_pylist() == expected
+
+
 def test_sequence_numpy_timestamp():
     data = [
         np.datetime64(datetime.datetime(2007, 7, 13, 1, 23, 34, 123456)),
@@ -837,34 +1032,6 @@ def test_sequence_numpy_timestamp():
                                                46, 57, 437699)
 
 
-def test_sequence_timestamp_with_unit():
-    data = [
-        datetime.datetime(2007, 7, 13, 1, 23, 34, 123456),
-    ]
-
-    s = pa.timestamp('s')
-    ms = pa.timestamp('ms')
-    us = pa.timestamp('us')
-
-    arr_s = pa.array(data, type=s)
-    assert len(arr_s) == 1
-    assert arr_s.type == s
-    assert arr_s[0].as_py() == datetime.datetime(2007, 7, 13, 1,
-                                                 23, 34, 0)
-
-    arr_ms = pa.array(data, type=ms)
-    assert len(arr_ms) == 1
-    assert arr_ms.type == ms
-    assert arr_ms[0].as_py() == datetime.datetime(2007, 7, 13, 1,
-                                                  23, 34, 123000)
-
-    arr_us = pa.array(data, type=us)
-    assert len(arr_us) == 1
-    assert arr_us.type == us
-    assert arr_us[0].as_py() == datetime.datetime(2007, 7, 13, 1,
-                                                  23, 34, 123456)
-
-
 class MyDate(datetime.date):
     pass
 
@@ -1421,17 +1588,6 @@ def test_decimal_array_with_none_and_nan():
     assert array.to_pylist() == [decimal.Decimal('1.2340'), None, None, None]
 
 
-@pytest.mark.parametrize('tz,name', [
-    (pytz.FixedOffset(90), '+01:30'),
-    (pytz.FixedOffset(-90), '-01:30'),
-    (pytz.utc, 'UTC'),
-    (pytz.timezone('America/New_York'), 'America/New_York')
-])
-def test_timezone_string(tz, name):
-    assert pa.lib.tzinfo_to_string(tz) == name
-    assert pa.lib.string_to_tzinfo(name) == tz
-
-
 def test_map_from_dicts():
     data = [[{'key': b'a', 'value': 1}, {'key': b'b', 'value': 2}],
             [{'key': b'c', 'value': 3}],
diff --git a/python/pyarrow/tests/test_pandas.py b/python/pyarrow/tests/test_pandas.py
index b023e39..2d66a32 100644
--- a/python/pyarrow/tests/test_pandas.py
+++ b/python/pyarrow/tests/test_pandas.py
@@ -22,7 +22,7 @@ import multiprocessing as mp
 import sys
 
 from collections import OrderedDict
-from datetime import date, datetime, time, timedelta
+from datetime import date, datetime, time, timedelta, timezone
 from distutils.version import LooseVersion
 
 import hypothesis as h
@@ -3327,13 +3327,31 @@ def test_cast_timestamp_unit():
     assert result.equals(expected)
 
 
-def test_struct_with_timestamp_tz():
+def test_nested_with_timestamp_tz_round_trip():
+    ts = pd.Timestamp.now()
+    ts_dt = ts.to_pydatetime()
+    arr = pa.array([ts_dt], type=pa.timestamp('us', tz='America/New_York'))
+    struct = pa.StructArray.from_arrays([arr, arr], ['start', 'stop'])
+
+    result = struct.to_pandas()
+    restored = pa.array(result)
+    assert restored.equals(struct)
+
+
+def test_nested_with_timestamp_tz():
     # ARROW-7723
     ts = pd.Timestamp.now()
+    ts_dt = ts.to_pydatetime()
 
     # XXX: Ensure that this data does not get promoted to nanoseconds (and thus
     # integers) to preserve behavior in 0.15.1
     for unit in ['s', 'ms', 'us']:
+        if unit in ['s', 'ms']:
+            # This is used for verifying timezone conversion to micros are not
+            # important
+            def truncate(x): return x.replace(microsecond=0)
+        else:
+            def truncate(x): return x
         arr = pa.array([ts], type=pa.timestamp(unit))
         arr2 = pa.array([ts], type=pa.timestamp(unit, tz='America/New_York'))
 
@@ -3342,20 +3360,30 @@ def test_struct_with_timestamp_tz():
 
         result = arr3.to_pandas()
         assert isinstance(result[0]['start'], datetime)
+        assert result[0]['start'].tzinfo is None
         assert isinstance(result[0]['stop'], datetime)
+        assert result[0]['stop'].tzinfo is None
 
         result = arr4.to_pandas()
         assert isinstance(result[0]['start'], datetime)
+        assert result[0]['start'].tzinfo is not None
+        utc_dt = result[0]['start'].astimezone(timezone.utc)
+        assert truncate(utc_dt).replace(tzinfo=None) == truncate(ts_dt)
         assert isinstance(result[0]['stop'], datetime)
+        assert result[0]['stop'].tzinfo is not None
 
         # same conversion for table
         result = pa.table({'a': arr3}).to_pandas()
         assert isinstance(result['a'][0]['start'], datetime)
+        assert result['a'][0]['start'].tzinfo is None
         assert isinstance(result['a'][0]['stop'], datetime)
+        assert result['a'][0]['stop'].tzinfo is None
 
         result = pa.table({'a': arr4}).to_pandas()
         assert isinstance(result['a'][0]['start'], datetime)
+        assert result['a'][0]['start'].tzinfo is not None
         assert isinstance(result['a'][0]['stop'], datetime)
+        assert result['a'][0]['stop'].tzinfo is not None
 
 
 # ----------------------------------------------------------------------
@@ -4032,19 +4060,25 @@ def test_timestamp_as_object_out_of_range():
 
 
 @pytest.mark.parametrize("resolution", ["s", "ms", "us"])
+@pytest.mark.parametrize("tz", [None, "America/New_York"])
 # One datetime outside nanosecond range, one inside nanosecond range:
 @pytest.mark.parametrize("dt", [datetime(1553, 1, 1), datetime(2020, 1, 1)])
-def test_timestamp_as_object_non_nanosecond(resolution, dt):
+def test_timestamp_as_object_non_nanosecond(resolution, tz, dt):
     # Timestamps can be converted Arrow and reloaded into Pandas with no loss
     # of information if the timestamp_as_object option is True.
-    arr = pa.array([dt], type=pa.timestamp(resolution))
-    result = arr.to_pandas(timestamp_as_object=True)
-    assert result.dtype == object
-    assert isinstance(result[0], datetime)
-    assert result[0] == dt
-
+    arr = pa.array([dt], type=pa.timestamp(resolution, tz=tz))
     table = pa.table({'a': arr})
-    result = table.to_pandas(timestamp_as_object=True)['a']
-    assert result.dtype == object
-    assert isinstance(result[0], datetime)
-    assert result[0] == dt
+
+    for result in [
+        arr.to_pandas(timestamp_as_object=True),
+        table.to_pandas(timestamp_as_object=True)['a']
+    ]:
+        assert result.dtype == object
+        assert isinstance(result[0], datetime)
+        if tz:
+            assert result[0].tzinfo is not None
+            expected = result[0].tzinfo.fromutc(dt)
+        else:
+            assert result[0].tzinfo is None
+            expected = dt
+        assert result[0] == expected
diff --git a/python/pyarrow/tests/test_types.py b/python/pyarrow/tests/test_types.py
index 1f905f3..c52751e 100644
--- a/python/pyarrow/tests/test_types.py
+++ b/python/pyarrow/tests/test_types.py
@@ -17,9 +17,12 @@
 
 from collections import OrderedDict
 from collections.abc import Iterator
+import datetime
+import sys
 
 import pickle
 import pytest
+import pytz
 import hypothesis as h
 import hypothesis.strategies as st
 import weakref
@@ -252,6 +255,120 @@ def test_is_primitive():
     assert not types.is_primitive(pa.list_(pa.int32()))
 
 
+@pytest.mark.parametrize(('tz', 'expected'), [
+    (pytz.utc, 'UTC'),
+    (pytz.timezone('Europe/Paris'), 'Europe/Paris'),
+    (pytz.FixedOffset(180), '+03:00'),
+    (datetime.timezone.utc, '+00:00'),
+    (datetime.timezone(datetime.timedelta(hours=1, minutes=30)), '+01:30')
+])
+def test_tzinfo_to_string(tz, expected):
+    assert pa.lib.tzinfo_to_string(tz) == expected
+
+
+def test_tzinfo_to_string_errors():
+    msg = "Not an instance of datetime.tzinfo"
+    with pytest.raises(TypeError):
+        pa.lib.tzinfo_to_string("Europe/Budapest")
+
+    if sys.version_info >= (3, 8):
+        # before 3.8 it was only possible to create timezone objects with whole
+        # number of minutes
+        tz = datetime.timezone(datetime.timedelta(hours=1, seconds=30))
+        msg = "Offset must represent whole number of minutes"
+        with pytest.raises(ValueError, match=msg):
+            pa.lib.tzinfo_to_string(tz)
+
+
+def test_convert_custom_tzinfo_objects_to_string():
+    class CorrectTimezone1(datetime.tzinfo):
+        """
+        Conversion is using utcoffset()
+        """
+
+        def tzname(self, dt):
+            return None
+
+        def utcoffset(self, dt):
+            return datetime.timedelta(hours=-3, minutes=30)
+
+    class CorrectTimezone2(datetime.tzinfo):
+        """
+        Conversion is using tzname()
+        """
+
+        def tzname(self, dt):
+            return "+03:00"
+
+        def utcoffset(self, dt):
+            return datetime.timedelta(hours=3)
+
+    class BuggyTimezone1(datetime.tzinfo):
+        """
+        Unable to infer name or offset
+        """
+
+        def tzname(self, dt):
+            return None
+
+        def utcoffset(self, dt):
+            return None
+
+    class BuggyTimezone2(datetime.tzinfo):
+        """
+        Wrong offset type
+        """
+
+        def tzname(self, dt):
+            return None
+
+        def utcoffset(self, dt):
+            return "one hour"
+
+    class BuggyTimezone3(datetime.tzinfo):
+        """
+        Wrong timezone name type
+        """
+
+        def tzname(self, dt):
+            return 240
+
+        def utcoffset(self, dt):
+            return None
+
+    assert pa.lib.tzinfo_to_string(CorrectTimezone1()) == "-02:30"
+    assert pa.lib.tzinfo_to_string(CorrectTimezone2()) == "+03:00"
+
+    msg = (r"Object returned by tzinfo.utcoffset\(None\) is not an instance "
+           r"of datetime.timedelta")
+    for wrong in [BuggyTimezone1(), BuggyTimezone2(), BuggyTimezone3()]:
+        with pytest.raises(ValueError, match=msg):
+            pa.lib.tzinfo_to_string(wrong)
+
+
+@pytest.mark.parametrize(('string', 'expected'), [
+    ('UTC', pytz.utc),
+    ('Europe/Paris', pytz.timezone('Europe/Paris')),
+    ('+03:00', pytz.FixedOffset(180)),
+    ('+01:30', pytz.FixedOffset(90)),
+    ('-02:00', pytz.FixedOffset(-120))
+])
+def test_string_to_tzinfo(string, expected):
+    result = pa.lib.string_to_tzinfo(string)
+    assert result == expected
+
+
+@pytest.mark.parametrize('tz,name', [
+    (pytz.FixedOffset(90), '+01:30'),
+    (pytz.FixedOffset(-90), '-01:30'),
+    (pytz.utc, 'UTC'),
+    (pytz.timezone('America/New_York'), 'America/New_York')
+])
+def test_timezone_string_roundtrip(tz, name):
+    assert pa.lib.tzinfo_to_string(tz) == name
+    assert pa.lib.string_to_tzinfo(name) == tz
+
+
 def test_timestamp():
     for unit in ('s', 'ms', 'us', 'ns'):
         for tz in (None, 'UTC', 'Europe/Paris'):
diff --git a/python/pyarrow/types.pxi b/python/pyarrow/types.pxi
index edd9622..15483d3 100644
--- a/python/pyarrow/types.pxi
+++ b/python/pyarrow/types.pxi
@@ -1816,9 +1816,6 @@ cdef timeunit_to_string(TimeUnit unit):
         return 'ns'
 
 
-_FIXED_OFFSET_RE = re.compile(r'([+-])(0[0-9]|1[0-9]|2[0-3]):([0-5][0-9])$')
-
-
 def tzinfo_to_string(tz):
     """
     Converts a time zone object into a string indicating the name of a time
@@ -1837,32 +1834,7 @@ def tzinfo_to_string(tz):
       name : str
         Time zone name
     """
-    import pytz
-    import datetime
-
-    def fixed_offset_to_string(offset):
-        seconds = int(offset.utcoffset(None).total_seconds())
-        sign = '+' if seconds >= 0 else '-'
-        minutes, seconds = divmod(abs(seconds), 60)
-        hours, minutes = divmod(minutes, 60)
-        if seconds > 0:
-            raise ValueError('Offset must represent whole number of minutes')
-        return '{}{:02d}:{:02d}'.format(sign, hours, minutes)
-
-    if tz is pytz.utc:
-        return tz.zone  # ARROW-4055
-    elif isinstance(tz, pytz.tzinfo.BaseTzInfo):
-        return tz.zone
-    elif isinstance(tz, pytz._FixedOffset):
-        return fixed_offset_to_string(tz)
-    elif isinstance(tz, datetime.tzinfo):
-        if isinstance(tz, datetime.timezone):
-            return fixed_offset_to_string(tz)
-        else:
-            raise ValueError('Unable to convert timezone `{}` to string'
-                             .format(tz))
-    else:
-        raise TypeError('Must be an instance of `datetime.tzinfo`')
+    return frombytes(GetResultValue(TzinfoToString(<PyObject*>tz)))
 
 
 def string_to_tzinfo(name):
@@ -1884,14 +1856,8 @@ def string_to_tzinfo(name):
       tz : datetime.tzinfo
         Time zone object
     """
-    import pytz
-    m = _FIXED_OFFSET_RE.match(name)
-    if m:
-        sign = 1 if m.group(1) == '+' else -1
-        hours, minutes = map(int, m.group(2, 3))
-        return pytz.FixedOffset(sign * (hours * 60 + minutes))
-    else:
-        return pytz.timezone(name)
+    cdef PyObject* tz = GetResultValue(StringToTzinfo(name.encode('utf-8')))
+    return PyObject_to_object(tz)
 
 
 def timestamp(unit, tz=None):