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/08/10 15:08:57 UTC

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

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