You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by GitBox <gi...@apache.org> on 2020/07/21 15:32:25 UTC

[GitHub] [arrow] kszucs opened a new pull request #7816: [Python] Better timezone support

kszucs opened a new pull request #7816:
URL: https://github.com/apache/arrow/pull/7816


   Follow up of:
   - ARROW-9223: [Python] Propagate timezone information in pandas conversion 
   - ARROW-9528: [Python] Honor tzinfo when converting from datetime


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] emkornfield commented on a change in pull request #7816: ARROW-9528: [Python] Honor tzinfo when converting from datetime

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #7816:
URL: https://github.com/apache/arrow/pull/7816#discussion_r467498668



##########
File path: 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 ARROW_NO_TZ=1

Review comment:
       how about ARROW_NO_TZ_LOCALIZATION or ARROW_NO_TZ_PRESERVATION?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] kszucs commented on a change in pull request #7816: ARROW-9528: [Python] Honor tzinfo when converting from datetime

Posted by GitBox <gi...@apache.org>.
kszucs commented on a change in pull request #7816:
URL: https://github.com/apache/arrow/pull/7816#discussion_r467967766



##########
File path: cpp/src/arrow/python/common.h
##########
@@ -137,6 +137,11 @@ class ARROW_PYTHON_EXPORT OwnedRef {
   OwnedRef(OwnedRef&& other) : OwnedRef(other.detach()) {}
   explicit OwnedRef(PyObject* obj) : obj_(obj) {}
 
+  OwnedRef& operator=(PyObject* obj) {
+    obj_ = obj;

Review comment:
       Removed the overload.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] kszucs commented on pull request #7816: ARROW-9528: [Python] Honor tzinfo when converting from datetime

Posted by GitBox <gi...@apache.org>.
kszucs commented on pull request #7816:
URL: https://github.com/apache/arrow/pull/7816#issuecomment-667630089


   Thanks Micah! 
   
   Testing it via the spark integration test SGTM. If we think that ignoring timezones during conversion could be a useful feature we could even expose it properly on the python side. 


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] emkornfield commented on a change in pull request #7816: ARROW-9528: [Python] Honor tzinfo when converting from datetime

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #7816:
URL: https://github.com/apache/arrow/pull/7816#discussion_r467353205



##########
File path: 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 ARROW_NO_TZ=1

Review comment:
       no, this also affects inference when importing non-naive datetimes  and when things like timestamp_as_object are used in to pandas.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] wesm commented on a change in pull request #7816: ARROW-9528: [Python] Honor tzinfo when converting from datetime

Posted by GitBox <gi...@apache.org>.
wesm commented on a change in pull request #7816:
URL: https://github.com/apache/arrow/pull/7816#discussion_r467500217



##########
File path: 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 ARROW_NO_TZ=1

Review comment:
       That works for me, perhaps "PYARROW_NO_TZ_PRESERVATION" (since it's Python-specific -- we have other PYARROW_-prefixed env vars) 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] emkornfield commented on a change in pull request #7816: ARROW-9528: [Python] Honor tzinfo when converting from datetime

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #7816:
URL: https://github.com/apache/arrow/pull/7816#discussion_r470941603



##########
File path: cpp/src/arrow/python/python_to_arrow.cc
##########
@@ -1328,6 +1357,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<TimestampType&>(*real_type);

Review comment:
       sorry, porbably just late, but not sure what you mean here.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] kszucs commented on a change in pull request #7816: ARROW-9528: [Python] Honor tzinfo when converting from datetime

Posted by GitBox <gi...@apache.org>.
kszucs commented on a change in pull request #7816:
URL: https://github.com/apache/arrow/pull/7816#discussion_r468021657



##########
File path: python/pyarrow/tests/test_types.py
##########
@@ -252,6 +255,121 @@ 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
+
+
+@pytest.mark.skipif(sys.version_info <= (3, 7), reason=(
+    "Since python 3.7 the UTC offset for datetime.timezone is not restricted "

Review comment:
       Removed the skip since we validate ourself that the timedelta has whole number of minutes.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] emkornfield commented on a change in pull request #7816: ARROW-9528: [Python] Honor tzinfo when converting from datetime

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #7816:
URL: https://github.com/apache/arrow/pull/7816#discussion_r470941434



##########
File path: cpp/src/arrow/python/datetime.cc
##########
@@ -262,6 +306,139 @@ 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, PyObject* pydelta_class) {
+  // attempt to convert timezone offset objects to "+/-{hh}:{mm}" format
+  OwnedRef pydelta_object(PyObject_CallMethod(pytzinfo, "utcoffset", "O", Py_None));
+  RETURN_IF_PYERROR();
+
+  if (!PyObject_IsInstance(pydelta_object.obj(), pydelta_class)) {
+    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();
+}
+
+// GIL must be held when calling this function.

Review comment:
       removed.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] kszucs commented on a change in pull request #7816: [Python] Better timezone support

Posted by GitBox <gi...@apache.org>.
kszucs commented on a change in pull request #7816:
URL: https://github.com/apache/arrow/pull/7816#discussion_r458859987



##########
File path: python/pyarrow/tests/test_convert_builtin.py
##########
@@ -811,6 +875,125 @@ 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_nanosecond_resolution_timestamp():

Review comment:
       I'm a bit uncertain whether we should support inferring/converting from `pandas.Timestamp` objects in the python builtin to arrow code paths since pandas is an optional dependency.
   
   If we leave it as is, the worst problem is that the nanosecond resolution `pandas.Timestamp` objects get truncated to microsecond resolution.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] wesm commented on a change in pull request #7816: ARROW-9528: [Python] Honor tzinfo when converting from datetime

Posted by GitBox <gi...@apache.org>.
wesm commented on a change in pull request #7816:
URL: https://github.com/apache/arrow/pull/7816#discussion_r467490629



##########
File path: 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 ARROW_NO_TZ=1

Review comment:
       OK, can you suggest a better name for this than "ARROW_NO_TZ"? It isn't clear to me what this option does from the name




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] github-actions[bot] commented on pull request #7816: ARROW-9528: [Python] Honor tzinfo when converting from datetime

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #7816:
URL: https://github.com/apache/arrow/pull/7816#issuecomment-667789181


   Revision: e27e86cdc83dede4bb70c49ac5af2fb3a0494c51
   
   Submitted crossbow builds: [ursa-labs/crossbow @ actions-455](https://github.com/ursa-labs/crossbow/branches/all?query=actions-455)
   
   |Task|Status|
   |----|------|
   |test-conda-python-3.8-spark-master|[![Github Actions](https://github.com/ursa-labs/crossbow/workflows/Crossbow/badge.svg?branch=actions-455-github-test-conda-python-3.8-spark-master)](https://github.com/ursa-labs/crossbow/actions?query=branch:actions-455-github-test-conda-python-3.8-spark-master)|


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] emkornfield commented on a change in pull request #7816: ARROW-9528: [Python] Honor tzinfo when converting from datetime

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #7816:
URL: https://github.com/apache/arrow/pull/7816#discussion_r467353299



##########
File path: cpp/src/arrow/python/python_to_arrow.cc
##########
@@ -191,10 +191,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*/) {

Review comment:
       My thought here was this parameter will only be need until the next release, so it would be better to add it here and then remove it without the need to introduce options.  We can introduce options though if you feel strongly.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] kszucs commented on pull request #7816: ARROW-9528: [Python] Honor tzinfo when converting from datetime

Posted by GitBox <gi...@apache.org>.
kszucs commented on pull request #7816:
URL: https://github.com/apache/arrow/pull/7816#issuecomment-667720694


   I mean ignore_tomezone could be a useful conversion option for mixed timezone aware/naive input values (should be rare or rather discouraged though). 
   
   I agree with the deprecation plan. I'm on vacation next week, but hopefully @pitrou and @jorisvandenbossche will be able to take a look at it.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] emkornfield commented on a change in pull request #7816: ARROW-9528: [Python] Honor tzinfo when converting from datetime

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #7816:
URL: https://github.com/apache/arrow/pull/7816#discussion_r470941271



##########
File path: cpp/src/arrow/python/arrow_to_pandas.cc
##########
@@ -642,24 +641,27 @@ 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.

Review comment:
       added: Nanoseconds are returned integers inside of structs.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] kszucs commented on a change in pull request #7816: ARROW-9528: [Python] Honor tzinfo when converting from datetime

Posted by GitBox <gi...@apache.org>.
kszucs commented on a change in pull request #7816:
URL: https://github.com/apache/arrow/pull/7816#discussion_r468021862



##########
File path: python/pyarrow/tests/test_types.py
##########
@@ -252,6 +255,121 @@ 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
+
+
+@pytest.mark.skipif(sys.version_info <= (3, 7), reason=(
+    "Since python 3.7 the UTC offset for datetime.timezone is not restricted "
+    "to a whole number of minutes"
+))
+def test_tzinfo_to_string_errors():
+    msg = "Offset must represent whole number of minutes"
+    with pytest.raises(ValueError, match=msg):
+        tz = datetime.timezone(datetime.timedelta(hours=1, seconds=30))
+        pa.lib.tzinfo_to_string(tz)
+
+    msg = "Not an instance of datetime.tzinfo"
+    with pytest.raises(ValueError):

Review comment:
       Updated.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] emkornfield commented on pull request #7816: ARROW-9528: [Python] Honor tzinfo when converting from datetime

Posted by GitBox <gi...@apache.org>.
emkornfield commented on pull request #7816:
URL: https://github.com/apache/arrow/pull/7816#issuecomment-667786442


   @github-actions crossbow submit test-conda-python-3.8-spark-master


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] kszucs commented on a change in pull request #7816: ARROW-9528: [Python] Honor tzinfo when converting from datetime

Posted by GitBox <gi...@apache.org>.
kszucs commented on a change in pull request #7816:
URL: https://github.com/apache/arrow/pull/7816#discussion_r467948354



##########
File path: 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()}) {

Review comment:
       Removed.

##########
File path: cpp/src/arrow/python/datetime.h
##########
@@ -157,6 +157,22 @@ 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);
+
+ARROW_PYTHON_EXPORT
+Result<std::string> TzinfoToString(PyObject* pytzinfo);

Review comment:
       Added.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] emkornfield commented on a change in pull request #7816: ARROW-9528: [Python] Honor tzinfo when converting from datetime

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #7816:
URL: https://github.com/apache/arrow/pull/7816#discussion_r463916695



##########
File path: cpp/src/arrow/python/python_to_arrow.cc
##########
@@ -240,19 +242,20 @@ struct ValueConverter<TimestampType> {
   static inline Result<int64_t> FromPython(PyObject* obj, TimeUnit::type unit) {
     int64_t value;
     if (PyDateTime_Check(obj)) {
+      ARROW_ASSIGN_OR_RAISE(auto offset, internal::PyDateTime_utcoffset_s(obj));

Review comment:
       can the type be spelled out instead of auto?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] emkornfield commented on pull request #7816: ARROW-9528: [Python] Honor tzinfo when converting from datetime

Posted by GitBox <gi...@apache.org>.
emkornfield commented on pull request #7816:
URL: https://github.com/apache/arrow/pull/7816#issuecomment-667632780


   I'll let others chime in but I think this PR captures correct behavior, so I'd like to deprecate backwards compatibility after a release or two


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] kszucs commented on a change in pull request #7816: ARROW-9528: [Python] Honor tzinfo when converting from datetime

Posted by GitBox <gi...@apache.org>.
kszucs commented on a change in pull request #7816:
URL: https://github.com/apache/arrow/pull/7816#discussion_r467992160



##########
File path: cpp/src/arrow/python/datetime.h
##########
@@ -157,6 +157,22 @@ inline int64_t PyDelta_to_ns(PyDateTime_Delta* pytimedelta) {
   return PyDelta_to_us(pytimedelta) * 1000;
 }
 
+ARROW_PYTHON_EXPORT

Review comment:
       Well, the other utility functions are exported as well.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] github-actions[bot] commented on pull request #7816: ARROW-9528: [Python] Honor tzinfo when converting from datetime

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #7816:
URL: https://github.com/apache/arrow/pull/7816#issuecomment-667816017


   Revision: b4e3b9d4dcf3da6c7fa3e652d5b77fc4e7abad13
   
   Submitted crossbow builds: [ursa-labs/crossbow @ actions-458](https://github.com/ursa-labs/crossbow/branches/all?query=actions-458)
   
   |Task|Status|
   |----|------|
   |test-conda-python-3.8-spark-master|[![Github Actions](https://github.com/ursa-labs/crossbow/workflows/Crossbow/badge.svg?branch=actions-458-github-test-conda-python-3.8-spark-master)](https://github.com/ursa-labs/crossbow/actions?query=branch:actions-458-github-test-conda-python-3.8-spark-master)|


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] emkornfield commented on a change in pull request #7816: ARROW-9528: [Python] Honor tzinfo when converting from datetime

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #7816:
URL: https://github.com/apache/arrow/pull/7816#discussion_r463915701



##########
File path: 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()}) {

Review comment:
       Is this required for this PR?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] emkornfield commented on a change in pull request #7816: ARROW-9528: [Python] Honor tzinfo when converting from datetime

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #7816:
URL: https://github.com/apache/arrow/pull/7816#discussion_r470941603



##########
File path: cpp/src/arrow/python/python_to_arrow.cc
##########
@@ -1328,6 +1357,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<TimestampType&>(*real_type);

Review comment:
       sorry, probably just late, but not sure what you mean here.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] emkornfield commented on pull request #7816: [Python] Better timezone support

Posted by GitBox <gi...@apache.org>.
emkornfield commented on pull request #7816:
URL: https://github.com/apache/arrow/pull/7816#issuecomment-661938939


   Thanks @kszucs per discussion I'm ok with this not making it into 1.0 release.  I am working on making moving tzinfo_to_string to c++ so we can accurately capture tz instead of going to utc.  I also still think a flag is a good idea to revert to buggy behavior.  Want me to post here?


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] emkornfield commented on a change in pull request #7816: ARROW-9528: [Python] Honor tzinfo when converting from datetime

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #7816:
URL: https://github.com/apache/arrow/pull/7816#discussion_r467532213



##########
File path: 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 ARROW_NO_TZ=1

Review comment:
       I went with PYARROW_IGNORE_TZ which I think is more succinct and I think still conveys the meaning.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] github-actions[bot] commented on pull request #7816: ARROW-9528: [Python] Honor tzinfo when converting from datetime

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #7816:
URL: https://github.com/apache/arrow/pull/7816#issuecomment-668170595


   Revision: b4e3b9d4dcf3da6c7fa3e652d5b77fc4e7abad13
   
   Submitted crossbow builds: [ursa-labs/crossbow @ actions-462](https://github.com/ursa-labs/crossbow/branches/all?query=actions-462)
   
   |Task|Status|
   |----|------|
   |test-conda-python-3.7-spark-branch-3.0|[![Github Actions](https://github.com/ursa-labs/crossbow/workflows/Crossbow/badge.svg?branch=actions-462-github-test-conda-python-3.7-spark-branch-3.0)](https://github.com/ursa-labs/crossbow/actions?query=branch:actions-462-github-test-conda-python-3.7-spark-branch-3.0)|


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] kszucs commented on pull request #7816: [Python] Better timezone support

Posted by GitBox <gi...@apache.org>.
kszucs commented on pull request #7816:
URL: https://github.com/apache/arrow/pull/7816#issuecomment-662420317


   @emkornfield I managed to implement TzinfoToString on the C++ side. Currently adding timezone inference support.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] kszucs commented on a change in pull request #7816: ARROW-9528: [Python] Honor tzinfo when converting from datetime

Posted by GitBox <gi...@apache.org>.
kszucs commented on a change in pull request #7816:
URL: https://github.com/apache/arrow/pull/7816#discussion_r468002813



##########
File path: cpp/src/arrow/python/datetime.cc
##########
@@ -262,6 +306,139 @@ 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, PyObject* pydelta_class) {
+  // attempt to convert timezone offset objects to "+/-{hh}:{mm}" format
+  OwnedRef pydelta_object(PyObject_CallMethod(pytzinfo, "utcoffset", "O", Py_None));
+  RETURN_IF_PYERROR();
+
+  if (!PyObject_IsInstance(pydelta_object.obj(), pydelta_class)) {

Review comment:
       Didn't know about the check functions, thanks!




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] emkornfield commented on a change in pull request #7816: ARROW-9528: [Python] Honor tzinfo when converting from datetime

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #7816:
URL: https://github.com/apache/arrow/pull/7816#discussion_r463916508



##########
File path: cpp/src/arrow/python/datetime.h
##########
@@ -157,6 +157,22 @@ 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);
+
+ARROW_PYTHON_EXPORT
+Result<std::string> TzinfoToString(PyObject* pytzinfo);

Review comment:
       Add docs here?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] kszucs commented on a change in pull request #7816: ARROW-9528: [Python] Honor tzinfo when converting from datetime

Posted by GitBox <gi...@apache.org>.
kszucs commented on a change in pull request #7816:
URL: https://github.com/apache/arrow/pull/7816#discussion_r467949355



##########
File path: 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 ARROW_NO_TZ=1

Review comment:
       I'd spell it out PYARROW_IGNORE_TIMEZONE




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] emkornfield commented on a change in pull request #7816: ARROW-9528: [Python] Honor tzinfo when converting from datetime

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #7816:
URL: https://github.com/apache/arrow/pull/7816#discussion_r463915789



##########
File path: cpp/src/arrow/python/arrow_to_pandas.cc
##########
@@ -642,24 +641,27 @@ 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.
   PandasOptions modified_options = options;
-  modified_options.ignore_timezone = true;
   modified_options.coerce_temporal_nanoseconds = false;
 
   for (int c = 0; c < data.num_chunks(); c++) {
     auto arr = checked_cast<const StructArray*>(data.chunk(c).get());
     // 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));
+      // Seen notes above about timstamp conversion.  Don't blindly convert because

Review comment:
       my code, I can fix.  Typos: "timestamp" and  "Seen"




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] emkornfield commented on a change in pull request #7816: ARROW-9528: [Python] Honor tzinfo when converting from datetime

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #7816:
URL: https://github.com/apache/arrow/pull/7816#discussion_r470940360



##########
File path: 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 ARROW_NO_TZ=1

Review comment:
       done.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] kszucs commented on a change in pull request #7816: ARROW-9528: [Python] Honor tzinfo when converting from datetime

Posted by GitBox <gi...@apache.org>.
kszucs commented on a change in pull request #7816:
URL: https://github.com/apache/arrow/pull/7816#discussion_r467968212



##########
File path: cpp/src/arrow/python/arrow_to_pandas.cc
##########
@@ -951,8 +953,21 @@ 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;
+    if (!type.timezone().empty() && !options.ignore_timezone) {
+      ARROW_ASSIGN_OR_RAISE(tzinfo, internal::StringToTzinfo(type.timezone()));
+      RETURN_IF_PYERROR();
+    }
+    auto WrapValue = [&](typename Type::c_type value, PyObject** out) {

Review comment:
       Updated.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] wesm commented on a change in pull request #7816: ARROW-9528: [Python] Honor tzinfo when converting from datetime

Posted by GitBox <gi...@apache.org>.
wesm commented on a change in pull request #7816:
URL: https://github.com/apache/arrow/pull/7816#discussion_r467330463



##########
File path: 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 ARROW_NO_TZ=1

Review comment:
       Nit: might be better to call this something more explicit like `PYARROW_PANDAS_DROP_NESTED_TZ` (since it only impacts tz-aware datetimes inside e.g. structs, right?)

##########
File path: cpp/src/arrow/python/arrow_to_pandas.cc
##########
@@ -642,24 +641,27 @@ 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.

Review comment:
       Might be worth stating again for emphasis that nanoseconds are returned as integers inside structs (I got confused myself when I saw it because I forgot about this)

##########
File path: cpp/src/arrow/python/python_to_arrow.cc
##########
@@ -191,10 +191,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*/) {

Review comment:
       This is a bit ugly, could we pass an options struct instead?

##########
File path: cpp/src/arrow/python/python_to_arrow.cc
##########
@@ -1328,6 +1357,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<TimestampType&>(*real_type);

Review comment:
       be consistent about const?

##########
File path: cpp/src/arrow/python/arrow_to_pandas.cc
##########
@@ -951,8 +953,21 @@ 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;
+    if (!type.timezone().empty() && !options.ignore_timezone) {
+      ARROW_ASSIGN_OR_RAISE(tzinfo, internal::StringToTzinfo(type.timezone()));
+      RETURN_IF_PYERROR();
+    }
+    auto WrapValue = [&](typename Type::c_type value, PyObject** out) {

Review comment:
       I'd suggest declaring two lambdas, one that is the original WrapValue and another that adds the tzinfo to the result of WrapValue

##########
File path: cpp/src/arrow/python/common.h
##########
@@ -137,6 +137,11 @@ class ARROW_PYTHON_EXPORT OwnedRef {
   OwnedRef(OwnedRef&& other) : OwnedRef(other.detach()) {}
   explicit OwnedRef(PyObject* obj) : obj_(obj) {}
 
+  OwnedRef& operator=(PyObject* obj) {
+    obj_ = obj;
+    return *this;
+  }

Review comment:
       Is this definitely needed? I think it's cleaner to use the explicit reset/detach methods

##########
File path: cpp/src/arrow/python/datetime.h
##########
@@ -157,6 +157,22 @@ inline int64_t PyDelta_to_ns(PyDateTime_Delta* pytimedelta) {
   return PyDelta_to_us(pytimedelta) * 1000;
 }
 
+ARROW_PYTHON_EXPORT

Review comment:
       Do these need to be exported?

##########
File path: 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()}) {

Review comment:
       I agree it's good to fix but not required for the pR

##########
File path: python/pyarrow/tests/test_pandas.py
##########
@@ -3325,13 +3325,35 @@ 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()
+    # N.B. we test with Pandas because the Arrow types are not

Review comment:
       Indeed it seems like `pa.array(result)` should recover `struct` and we want to verify that?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] emkornfield commented on pull request #7816: ARROW-9528: [Python] Honor tzinfo when converting from datetime

Posted by GitBox <gi...@apache.org>.
emkornfield commented on pull request #7816:
URL: https://github.com/apache/arrow/pull/7816#issuecomment-668169810


   @github-actions crossbow submit test-conda-python-3.7-spark-branch-3.0


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] emkornfield commented on pull request #7816: ARROW-9528: [Python] Honor tzinfo when converting from datetime

Posted by GitBox <gi...@apache.org>.
emkornfield commented on pull request #7816:
URL: https://github.com/apache/arrow/pull/7816#issuecomment-667624552


   @kszucs I pushed a few commits to:
   1.  Fix typos I found on rereview.
   2.  Plumb through turning off the use of TZ informaiton in C++ via options.
   3.  Population in python via ARROW_NO_TZ environment variable
   
   I'd appreciate any thoughts you have on testing the backwards compatibility (maybe it is sufficient to use this with spark integration tests?)


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] emkornfield commented on a change in pull request #7816: ARROW-9528: [Python] Honor tzinfo when converting from datetime

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #7816:
URL: https://github.com/apache/arrow/pull/7816#discussion_r463916946



##########
File path: python/pyarrow/tests/test_types.py
##########
@@ -251,6 +254,121 @@ 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
+
+
+@pytest.mark.skipif(sys.version_info <= (3, 7), reason=(
+    "Since python 3.7 the UTC offset for datetime.timezone is not restricted "
+    "to a whole number of minutes"
+))
+def test_tzinfo_to_string_errors():
+    msg = "Offset must represent whole number of minutes"
+    with pytest.raises(ValueError, match=msg):
+        tz = datetime.timezone(datetime.timedelta(hours=1, seconds=30))
+        pa.lib.tzinfo_to_string(tz)
+
+    msg = "Not an instance of datetime.tzinfo"
+    with pytest.raises(ValueError):
+        pa.lib.tzinfo_to_string("Europe/Budapest")
+
+
+def test_convert_custom_tzinfo_objects_to_string():

Review comment:
       thank you for the thorough tests.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] emkornfield commented on a change in pull request #7816: ARROW-9528: [Python] Honor tzinfo when converting from datetime

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #7816:
URL: https://github.com/apache/arrow/pull/7816#discussion_r463917731



##########
File path: cpp/src/arrow/python/common.h
##########
@@ -137,6 +137,11 @@ class ARROW_PYTHON_EXPORT OwnedRef {
   OwnedRef(OwnedRef&& other) : OwnedRef(other.detach()) {}
   explicit OwnedRef(PyObject* obj) : obj_(obj) {}
 
+  OwnedRef& operator=(PyObject* obj) {
+    obj_ = obj;

Review comment:
       should there be a DecRef here?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] emkornfield commented on pull request #7816: ARROW-9528: [Python] Honor tzinfo when converting from datetime

Posted by GitBox <gi...@apache.org>.
emkornfield commented on pull request #7816:
URL: https://github.com/apache/arrow/pull/7816#issuecomment-674353756


   @wesm @kszucs thoughts on any remaining items on this?


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] wesm closed pull request #7816: ARROW-9528: [Python] Honor tzinfo when converting from datetime

Posted by GitBox <gi...@apache.org>.
wesm closed pull request #7816:
URL: https://github.com/apache/arrow/pull/7816


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] kszucs commented on pull request #7816: [Python] Better timezone support

Posted by GitBox <gi...@apache.org>.
kszucs commented on pull request #7816:
URL: https://github.com/apache/arrow/pull/7816#issuecomment-661956856


   @emkornfield sure, please feel free to push to this PR. 


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] kszucs edited a comment on pull request #7816: [Python] Better timezone support

Posted by GitBox <gi...@apache.org>.
kszucs edited a comment on pull request #7816:
URL: https://github.com/apache/arrow/pull/7816#issuecomment-661956856


   @emkornfield sure, please feel free to push to this PR. I'm going to work on the Time type support then.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] kszucs commented on pull request #7816: [Python] Better timezone support

Posted by GitBox <gi...@apache.org>.
kszucs commented on pull request #7816:
URL: https://github.com/apache/arrow/pull/7816#issuecomment-661933990


   @emkornfield I've cherry picked ARROW-9223, ARROW-9528, addressed Antoine's reviews comments and simplified the conversion Timestamp conversion path a bit. I'm going to add timezone support for the Time types as well.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] kszucs commented on a change in pull request #7816: ARROW-9528: [Python] Honor tzinfo when converting from datetime

Posted by GitBox <gi...@apache.org>.
kszucs commented on a change in pull request #7816:
URL: https://github.com/apache/arrow/pull/7816#discussion_r467892195



##########
File path: cpp/src/arrow/python/common.h
##########
@@ -137,6 +137,11 @@ class ARROW_PYTHON_EXPORT OwnedRef {
   OwnedRef(OwnedRef&& other) : OwnedRef(other.detach()) {}
   explicit OwnedRef(PyObject* obj) : obj_(obj) {}
 
+  OwnedRef& operator=(PyObject* obj) {
+    obj_ = obj;
+    return *this;
+  }

Review comment:
       Agreed, removing.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] wesm commented on pull request #7816: ARROW-9528: [Python] Honor tzinfo when converting from datetime

Posted by GitBox <gi...@apache.org>.
wesm commented on pull request #7816:
URL: https://github.com/apache/arrow/pull/7816#issuecomment-674421373


   Sorry for the delay, didn't have a lot of time for development this week. I am going to pull this down and kick the tires a bit this weekend and will merge later today or tomorrow


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] github-actions[bot] commented on pull request #7816: ARROW-9528: [Python] Honor tzinfo when converting from datetime

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #7816:
URL: https://github.com/apache/arrow/pull/7816#issuecomment-662541582


   https://issues.apache.org/jira/browse/ARROW-9528


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] emkornfield commented on pull request #7816: ARROW-9528: [Python] Honor tzinfo when converting from datetime

Posted by GitBox <gi...@apache.org>.
emkornfield commented on pull request #7816:
URL: https://github.com/apache/arrow/pull/7816#issuecomment-667815553


   @github-actions crossbow submit test-conda-python-3.8-spark-master


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] kszucs commented on a change in pull request #7816: [Python] Better timezone support

Posted by GitBox <gi...@apache.org>.
kszucs commented on a change in pull request #7816:
URL: https://github.com/apache/arrow/pull/7816#discussion_r458859987



##########
File path: python/pyarrow/tests/test_convert_builtin.py
##########
@@ -811,6 +875,125 @@ 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_nanosecond_resolution_timestamp():

Review comment:
       I'm a bit uncertain whether we should support inferring/converting from `pandas.Timestamp` objects in the python builtin to arrow code paths since pandas is an optional dependency.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] pitrou commented on a change in pull request #7816: ARROW-9528: [Python] Honor tzinfo when converting from datetime

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #7816:
URL: https://github.com/apache/arrow/pull/7816#discussion_r467961929



##########
File path: cpp/src/arrow/python/datetime.cc
##########
@@ -262,6 +306,139 @@ 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, PyObject* pydelta_class) {
+  // attempt to convert timezone offset objects to "+/-{hh}:{mm}" format
+  OwnedRef pydelta_object(PyObject_CallMethod(pytzinfo, "utcoffset", "O", Py_None));
+  RETURN_IF_PYERROR();
+
+  if (!PyObject_IsInstance(pydelta_object.obj(), pydelta_class)) {
+    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();
+}
+
+// GIL must be held when calling this function.

Review comment:
       This is true for other functions as well, so why mention it here?

##########
File path: python/pyarrow/tests/test_types.py
##########
@@ -252,6 +255,121 @@ 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
+
+
+@pytest.mark.skipif(sys.version_info <= (3, 7), reason=(
+    "Since python 3.7 the UTC offset for datetime.timezone is not restricted "
+    "to a whole number of minutes"
+))
+def test_tzinfo_to_string_errors():
+    msg = "Offset must represent whole number of minutes"
+    with pytest.raises(ValueError, match=msg):
+        tz = datetime.timezone(datetime.timedelta(hours=1, seconds=30))
+        pa.lib.tzinfo_to_string(tz)
+
+    msg = "Not an instance of datetime.tzinfo"
+    with pytest.raises(ValueError):

Review comment:
       Shouldn't this be `TypeError`?

##########
File path: cpp/src/arrow/python/arrow_to_pandas.cc
##########
@@ -950,13 +952,38 @@ struct ObjectWriterVisitor {
 
   template <typename Type>
   enable_if_timestamp<Type, Status> Visit(const Type& type) {
+    PyObject* tzinfo = nullptr;
     const TimeUnit::type unit = type.unit();
-    auto WrapValue = [unit](typename Type::c_type value, PyObject** out) {
+
+    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* tz_naive;

Review comment:
       If `tz_naive` really a timezone? AFAIK, it's a datetime, so call it e.g. `datetime_naive`.

##########
File path: cpp/src/arrow/python/datetime.cc
##########
@@ -262,6 +306,139 @@ 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, PyObject* pydelta_class) {
+  // attempt to convert timezone offset objects to "+/-{hh}:{mm}" format
+  OwnedRef pydelta_object(PyObject_CallMethod(pytzinfo, "utcoffset", "O", Py_None));
+  RETURN_IF_PYERROR();
+
+  if (!PyObject_IsInstance(pydelta_object.obj(), pydelta_class)) {

Review comment:
       You can use `PyDelta_Check`. See https://docs.python.org/3/c-api/datetime.html

##########
File path: cpp/src/arrow/python/datetime.cc
##########
@@ -262,6 +306,139 @@ 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, PyObject* pydelta_class) {
+  // attempt to convert timezone offset objects to "+/-{hh}:{mm}" format
+  OwnedRef pydelta_object(PyObject_CallMethod(pytzinfo, "utcoffset", "O", Py_None));
+  RETURN_IF_PYERROR();
+
+  if (!PyObject_IsInstance(pydelta_object.obj(), pydelta_class)) {
+    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();
+}
+
+// GIL must be held when calling this function.
+// 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_tzinfo;       // from datetime import tzinfo
+  OwnedRef class_timezone;     // from datetime import timezone
+  OwnedRef class_timedelta;    // from datetime import timedelta
+  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(), "tzinfo", &class_tzinfo));
+  RETURN_NOT_OK(
+      internal::ImportFromModule(module_datetime.obj(), "timezone", &class_timezone));
+  RETURN_NOT_OK(
+      internal::ImportFromModule(module_datetime.obj(), "timedelta", &class_timedelta));
+
+  // check that it's a valid tzinfo object
+  if (!PyObject_IsInstance(tzinfo, class_tzinfo.obj())) {

Review comment:
       `PyTZInfo_Check`?

##########
File path: cpp/src/arrow/python/arrow_to_pandas.cc
##########
@@ -950,13 +952,38 @@ struct ObjectWriterVisitor {
 
   template <typename Type>
   enable_if_timestamp<Type, Status> Visit(const Type& type) {
+    PyObject* tzinfo = nullptr;
     const TimeUnit::type unit = type.unit();
-    auto WrapValue = [unit](typename Type::c_type value, PyObject** out) {
+
+    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* tz_naive;
+      RETURN_NOT_OK(ConvertTimezoneNaive(value, &tz_naive));
+      // convert the timezone naive datetime object to timezone aware
+      *out = PyObject_CallMethod(tzinfo, "fromutc", "O", tz_naive);
+      RETURN_IF_PYERROR();
+      // the timezone naive object is no longer required
+      Py_DECREF(tz_naive);
+      return Status::OK();
+    };
+
+    if (!type.timezone().empty() && !options.ignore_timezone) {
+      // convert timezone aware
+      ARROW_ASSIGN_OR_RAISE(tzinfo, internal::StringToTzinfo(type.timezone()));

Review comment:
       You should `Py_DECREF(tzinfo)` at some point (or use a `OwnedRef`).

##########
File path: cpp/src/arrow/python/arrow_to_pandas.cc
##########
@@ -950,13 +952,38 @@ struct ObjectWriterVisitor {
 
   template <typename Type>
   enable_if_timestamp<Type, Status> Visit(const Type& type) {
+    PyObject* tzinfo = nullptr;
     const TimeUnit::type unit = type.unit();
-    auto WrapValue = [unit](typename Type::c_type value, PyObject** out) {
+
+    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* tz_naive;
+      RETURN_NOT_OK(ConvertTimezoneNaive(value, &tz_naive));
+      // convert the timezone naive datetime object to timezone aware
+      *out = PyObject_CallMethod(tzinfo, "fromutc", "O", tz_naive);
+      RETURN_IF_PYERROR();
+      // the timezone naive object is no longer required
+      Py_DECREF(tz_naive);

Review comment:
       You must call this before `RETURN_IF_PYERROR`.

##########
File path: python/pyarrow/tests/test_types.py
##########
@@ -252,6 +255,121 @@ 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
+
+
+@pytest.mark.skipif(sys.version_info <= (3, 7), reason=(
+    "Since python 3.7 the UTC offset for datetime.timezone is not restricted "

Review comment:
       Based on this message, I don't understand why you skip for `sys.version_info <= (3, 7)` rather than `sys.version_info >= (3, 7)`.

##########
File path: cpp/src/arrow/python/datetime.cc
##########
@@ -262,6 +306,139 @@ 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, PyObject* pydelta_class) {
+  // attempt to convert timezone offset objects to "+/-{hh}:{mm}" format
+  OwnedRef pydelta_object(PyObject_CallMethod(pytzinfo, "utcoffset", "O", Py_None));
+  RETURN_IF_PYERROR();
+
+  if (!PyObject_IsInstance(pydelta_object.obj(), pydelta_class)) {
+    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();
+}
+
+// GIL must be held when calling this function.
+// 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_tzinfo;       // from datetime import tzinfo
+  OwnedRef class_timezone;     // from datetime import timezone
+  OwnedRef class_timedelta;    // from datetime import timedelta
+  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(), "tzinfo", &class_tzinfo));
+  RETURN_NOT_OK(
+      internal::ImportFromModule(module_datetime.obj(), "timezone", &class_timezone));
+  RETURN_NOT_OK(
+      internal::ImportFromModule(module_datetime.obj(), "timedelta", &class_timedelta));
+
+  // check that it's a valid tzinfo object
+  if (!PyObject_IsInstance(tzinfo, class_tzinfo.obj())) {
+    return Status::Invalid("Not an instance of datetime.tzinfo");
+  }
+
+  // if tzinfo is an instance of pytz._FixedOffset or datetime.timedelta return the

Review comment:
       "or datetime.timezone"




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] emkornfield commented on a change in pull request #7816: ARROW-9528: [Python] Honor tzinfo when converting from datetime

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #7816:
URL: https://github.com/apache/arrow/pull/7816#discussion_r463916817



##########
File path: python/pyarrow/tests/test_pandas.py
##########
@@ -3325,13 +3325,35 @@ 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()
+    # N.B. we test with Pandas because the Arrow types are not

Review comment:
       this might not be true anymore with StringToTz?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] kszucs commented on a change in pull request #7816: ARROW-9528: [Python] Honor tzinfo when converting from datetime

Posted by GitBox <gi...@apache.org>.
kszucs commented on a change in pull request #7816:
URL: https://github.com/apache/arrow/pull/7816#discussion_r467887919



##########
File path: 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()}) {

Review comment:
       It wasn't compiling for me with llvm 10.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org