You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by we...@apache.org on 2018/11/27 22:16:18 UTC

[arrow] branch master updated: ARROW-3790: [C++] Fix erroneous safe casting

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

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


The following commit(s) were added to refs/heads/master by this push:
     new c98399d  ARROW-3790: [C++] Fix erroneous safe casting
c98399d is described below

commit c98399d500e0d3db47987e826af88c7376eed7c4
Author: François Saint-Jacques <fs...@gmail.com>
AuthorDate: Tue Nov 27 16:16:09 2018 -0600

    ARROW-3790: [C++] Fix erroneous safe casting
    
    - Improves coverage of various type combinations under safe/unsafe
      conditions
    - Fixes proper bound detection at compile time
    - Fixes type traits pattern matching
    
    While reviewing this merge, I noted that python, Ruby/glib and R would use CastOptions differently.
    
    - Python only exposes safe as a single boolean (which flips all options accordingly)
    - Ruby exposes 2 out of 3 of the options (probably due to code rot)
    - R is going to exposes all 3 modes.
    
    What is the purpose of having the 3 options, could we just expose safe versus unsafe?
    
    Author: François Saint-Jacques <fs...@gmail.com>
    
    Closes #3022 from fsaintjacques/ARROW-3790-unsigned-cast and squashes the following commits:
    
    a54392831 <François Saint-Jacques> ARROW-3790:  Fix erroneous safe casting
---
 cpp/src/arrow/compute/compute-test.cc | 115 ++++++++++++++++++++
 cpp/src/arrow/compute/kernels/cast.cc | 197 ++++++++++++++++++++++++++++------
 2 files changed, 277 insertions(+), 35 deletions(-)

diff --git a/cpp/src/arrow/compute/compute-test.cc b/cpp/src/arrow/compute/compute-test.cc
index ea91021..821569e 100644
--- a/cpp/src/arrow/compute/compute-test.cc
+++ b/cpp/src/arrow/compute/compute-test.cc
@@ -236,6 +236,76 @@ TEST_F(TestCast, ToIntDowncastSafe) {
   // underflow
   vector<int32_t> v6 = {0, 1000, 2000, -70000, 0};
   CheckFails<Int32Type>(int32(), v6, is_valid, int16(), options);
+
+  vector<int32_t> v7 = {0, 1000, 2000, -70000, 0};
+  CheckFails<Int32Type>(int32(), v7, is_valid, uint8(), options);
+}
+
+template <typename O, typename I>
+std::vector<O> UnsafeVectorCast(const std::vector<I>& v) {
+  size_t n_elems = v.size();
+  std::vector<O> result(n_elems);
+
+  for (size_t i = 0; i < v.size(); i++) result[i] = static_cast<O>(v[i]);
+
+  return std::move(result);
+}
+
+TEST_F(TestCast, IntegerSignedToUnsigned) {
+  CastOptions options;
+  options.allow_int_overflow = false;
+
+  vector<bool> is_valid = {true, false, true, true, true};
+
+  vector<int32_t> v1 = {INT32_MIN, 100, -1, UINT16_MAX, INT32_MAX};
+
+  // Same width
+  CheckFails<Int32Type>(int32(), v1, is_valid, uint32(), options);
+  // Wider
+  CheckFails<Int32Type>(int32(), v1, is_valid, uint64(), options);
+  // Narrower
+  CheckFails<Int32Type>(int32(), v1, is_valid, uint16(), options);
+  // Fail because of overflow (instead of underflow).
+  vector<int32_t> over = {0, -11, 0, UINT16_MAX + 1, INT32_MAX};
+  CheckFails<Int32Type>(int32(), over, is_valid, uint16(), options);
+
+  options.allow_int_overflow = true;
+
+  CheckCase<Int32Type, int32_t, UInt32Type, uint32_t>(
+      int32(), v1, is_valid, uint32(), UnsafeVectorCast<uint32_t, int32_t>(v1), options);
+  CheckCase<Int32Type, int32_t, UInt64Type, uint64_t>(
+      int32(), v1, is_valid, uint64(), UnsafeVectorCast<uint64_t, int32_t>(v1), options);
+  CheckCase<Int32Type, int32_t, UInt16Type, uint16_t>(
+      int32(), v1, is_valid, uint16(), UnsafeVectorCast<uint16_t, int32_t>(v1), options);
+  CheckCase<Int32Type, int32_t, UInt16Type, uint16_t>(
+      int32(), over, is_valid, uint16(), UnsafeVectorCast<uint16_t, int32_t>(over),
+      options);
+}
+
+TEST_F(TestCast, IntegerUnsignedToSigned) {
+  CastOptions options;
+  options.allow_int_overflow = false;
+
+  vector<bool> is_valid = {true, true, true};
+
+  vector<uint32_t> v1 = {0, INT16_MAX + 1, UINT32_MAX};
+  vector<uint32_t> v2 = {0, INT16_MAX + 1, 2};
+  // Same width
+  CheckFails<UInt32Type>(uint32(), v1, is_valid, int32(), options);
+  // Narrower
+  CheckFails<UInt32Type>(uint32(), v1, is_valid, int16(), options);
+  CheckFails<UInt32Type>(uint32(), v2, is_valid, int16(), options);
+
+  options.allow_int_overflow = true;
+
+  CheckCase<UInt32Type, uint32_t, Int32Type, int32_t>(
+      uint32(), v1, is_valid, int32(), UnsafeVectorCast<int32_t, uint32_t>(v1), options);
+  CheckCase<UInt32Type, uint32_t, Int64Type, int64_t>(
+      uint32(), v1, is_valid, int64(), UnsafeVectorCast<int64_t, uint32_t>(v1), options);
+  CheckCase<UInt32Type, uint32_t, Int16Type, int16_t>(
+      uint32(), v1, is_valid, int16(), UnsafeVectorCast<int16_t, uint32_t>(v1), options);
+  CheckCase<UInt32Type, uint32_t, Int16Type, int16_t>(
+      uint32(), v2, is_valid, int16(), UnsafeVectorCast<int16_t, uint32_t>(v2), options);
 }
 
 TEST_F(TestCast, ToIntDowncastUnsafe) {
@@ -343,6 +413,21 @@ TEST_F(TestCast, FloatingPointToInt) {
                                                     options);
 }
 
+TEST_F(TestCast, IntToFloatingPoint) {
+  auto options = CastOptions::Safe();
+
+  vector<bool> all_valid = {true, true, true, true, true};
+  vector<bool> all_invalid = {false, false, false, false, false};
+
+  vector<int64_t> v1 = {INT64_MIN, INT64_MIN + 1, 0, INT64_MAX - 1, INT64_MAX};
+  CheckFails<Int64Type>(int64(), v1, all_valid, float32(), options);
+
+  // While it's not safe to convert, all values are null.
+  CheckCase<Int64Type, int64_t, DoubleType, double>(int64(), v1, all_invalid, float64(),
+                                                    UnsafeVectorCast<double, int64_t>(v1),
+                                                    options);
+}
+
 TEST_F(TestCast, TimestampToTimestamp) {
   CastOptions options;
 
@@ -583,6 +668,36 @@ TEST_F(TestCast, TimeToCompatible) {
                          options);
 }
 
+TEST_F(TestCast, PrimitiveZeroCopy) {
+  shared_ptr<Array> arr;
+
+  ArrayFromVector<UInt8Type, uint8_t>(uint8(), {1, 1, 1, 1}, {1, 2, 3, 4}, &arr);
+  CheckZeroCopy(*arr, uint8());
+  ArrayFromVector<Int8Type, int8_t>(int8(), {1, 1, 1, 1}, {1, 2, 3, 4}, &arr);
+  CheckZeroCopy(*arr, int8());
+
+  ArrayFromVector<UInt16Type, uint16_t>(uint16(), {1, 1, 1, 1}, {1, 2, 3, 4}, &arr);
+  CheckZeroCopy(*arr, uint16());
+  ArrayFromVector<Int16Type, int8_t>(int16(), {1, 1, 1, 1}, {1, 2, 3, 4}, &arr);
+  CheckZeroCopy(*arr, int16());
+
+  ArrayFromVector<UInt32Type, uint32_t>(uint32(), {1, 1, 1, 1}, {1, 2, 3, 4}, &arr);
+  CheckZeroCopy(*arr, uint32());
+  ArrayFromVector<Int32Type, int8_t>(int32(), {1, 1, 1, 1}, {1, 2, 3, 4}, &arr);
+  CheckZeroCopy(*arr, int32());
+
+  ArrayFromVector<UInt64Type, uint64_t>(uint64(), {1, 1, 1, 1}, {1, 2, 3, 4}, &arr);
+  CheckZeroCopy(*arr, uint64());
+  ArrayFromVector<Int64Type, int8_t>(int64(), {1, 1, 1, 1}, {1, 2, 3, 4}, &arr);
+  CheckZeroCopy(*arr, int64());
+
+  ArrayFromVector<FloatType, float>(float32(), {1, 1, 1, 1}, {1, 2, 3, 4}, &arr);
+  CheckZeroCopy(*arr, float32());
+
+  ArrayFromVector<DoubleType, double>(float64(), {1, 1, 1, 1}, {1, 2, 3, 4}, &arr);
+  CheckZeroCopy(*arr, float64());
+}
+
 TEST_F(TestCast, DateToCompatible) {
   CastOptions options;
 
diff --git a/cpp/src/arrow/compute/kernels/cast.cc b/cpp/src/arrow/compute/kernels/cast.cc
index 97bc141..cd45b2d 100644
--- a/cpp/src/arrow/compute/kernels/cast.cc
+++ b/cpp/src/arrow/compute/kernels/cast.cc
@@ -89,6 +89,8 @@ template <typename O, typename I>
 struct is_zero_copy_cast<
     O, I,
     typename std::enable_if<std::is_same<I, O>::value &&
+                            // Parametric types contains runtime data which
+                            // differentiate them, it cannot be checked statically.
                             !std::is_base_of<ParametricType, O>::value>::type> {
   static constexpr bool value = true;
 };
@@ -161,23 +163,62 @@ struct CastFunctor<T, BooleanType, enable_if_number<T>> {
   }
 };
 
+// Number to Boolean
+template <typename I>
+struct CastFunctor<BooleanType, I,
+                   typename std::enable_if<std::is_base_of<Number, I>::value &&
+                                           !std::is_same<BooleanType, I>::value>::type> {
+  void operator()(FunctionContext* ctx, const CastOptions& options,
+                  const ArrayData& input, ArrayData* output) {
+    auto in_data = input.GetValues<typename I::c_type>(1);
+    const auto generate = [&in_data]() -> bool { return *in_data++ != 0; };
+    internal::GenerateBitsUnrolled(output->buffers[1]->mutable_data(), output->offset,
+                                   input.length, generate);
+  }
+};
+
 // ----------------------------------------------------------------------
 // Integers and Floating Point
 
+// Conversions pairs (<O, I>) are partitioned in 4 type traits:
+// - is_number_downcast
+// - is_integral_signed_to_unsigned
+// - is_integral_unsigned_to_signed
+// - is_float_truncate
+//
+// Each class has a different way of validation if the conversion is safe
+// (either with bounded intervals or with explicit C casts)
+
+template <typename O, typename I, typename Enable = void>
+struct is_number_downcast {
+  static constexpr bool value = false;
+};
+
 template <typename O, typename I>
-struct is_numeric_cast {
+struct is_number_downcast<
+    O, I,
+    typename std::enable_if<std::is_base_of<Number, O>::value &&
+                            std::is_base_of<Number, I>::value>::type> {
+  using O_T = typename O::c_type;
+  using I_T = typename I::c_type;
+
   static constexpr bool value =
-      (std::is_base_of<Number, O>::value && std::is_base_of<Number, I>::value) &&
-      (!std::is_same<O, I>::value);
+      ((!std::is_same<O, I>::value) &&
+       // Both types are of the same sign-ness.
+       ((std::is_signed<O_T>::value == std::is_signed<I_T>::value) &&
+        // Both types are of the same integral-ness.
+        (std::is_floating_point<O_T>::value == std::is_floating_point<I_T>::value)) &&
+       // Smaller output size
+       (sizeof(O_T) < sizeof(I_T)));
 };
 
 template <typename O, typename I, typename Enable = void>
-struct is_integer_downcast {
+struct is_integral_signed_to_unsigned {
   static constexpr bool value = false;
 };
 
 template <typename O, typename I>
-struct is_integer_downcast<
+struct is_integral_signed_to_unsigned<
     O, I,
     typename std::enable_if<std::is_base_of<Integer, O>::value &&
                             std::is_base_of<Integer, I>::value>::type> {
@@ -186,45 +227,92 @@ struct is_integer_downcast<
 
   static constexpr bool value =
       ((!std::is_same<O, I>::value) &&
-
-       // same size, but unsigned to signed
-       ((sizeof(O_T) == sizeof(I_T) && std::is_signed<O_T>::value &&
-         std::is_unsigned<I_T>::value) ||
-
-        // Smaller output size
-        (sizeof(O_T) < sizeof(I_T))));
+       ((std::is_unsigned<O_T>::value && std::is_signed<I_T>::value)));
 };
 
 template <typename O, typename I, typename Enable = void>
-struct is_float_truncate {
+struct is_integral_unsigned_to_signed {
   static constexpr bool value = false;
 };
 
 template <typename O, typename I>
-struct is_float_truncate<
+struct is_integral_unsigned_to_signed<
     O, I,
     typename std::enable_if<std::is_base_of<Integer, O>::value &&
-                            std::is_base_of<FloatingPoint, I>::value>::type> {
-  static constexpr bool value = true;
+                            std::is_base_of<Integer, I>::value>::type> {
+  using O_T = typename O::c_type;
+  using I_T = typename I::c_type;
+
+  static constexpr bool value =
+      ((!std::is_same<O, I>::value) &&
+       ((std::is_signed<O_T>::value && std::is_unsigned<I_T>::value)));
 };
 
+// This set of functions SafeMinimum/SafeMaximum would be simplified with
+// C++17 and `if constexpr`.
+
+// clang-format doesn't handle this construct properly. Thus the macro, but it
+// also improves readability.
+//
+// The effective return type of the function is always `I::c_type`, this is
+// just how enable_if works with functions.
+#define RET_TYPE(TRAIT) \
+  typename std::enable_if<TRAIT<O, I>::value, typename I::c_type>::type
+
 template <typename O, typename I>
-struct CastFunctor<O, I,
-                   typename std::enable_if<std::is_same<BooleanType, O>::value &&
-                                           std::is_base_of<Number, I>::value &&
-                                           !std::is_same<O, I>::value>::type> {
-  void operator()(FunctionContext* ctx, const CastOptions& options,
-                  const ArrayData& input, ArrayData* output) {
-    auto in_data = input.GetValues<typename I::c_type>(1);
-    const auto generate = [&in_data]() -> bool { return *in_data++ != 0; };
-    internal::GenerateBitsUnrolled(output->buffers[1]->mutable_data(), output->offset,
-                                   input.length, generate);
-  }
-};
+constexpr RET_TYPE(is_number_downcast) SafeMinimum() {
+  using out_type = typename O::c_type;
+
+  return std::numeric_limits<out_type>::lowest();
+}
 
 template <typename O, typename I>
-struct CastFunctor<O, I,
-                   typename std::enable_if<is_integer_downcast<O, I>::value>::type> {
+constexpr RET_TYPE(is_number_downcast) SafeMaximum() {
+  using out_type = typename O::c_type;
+
+  return std::numeric_limits<out_type>::max();
+}
+
+template <typename O, typename I>
+constexpr RET_TYPE(is_integral_unsigned_to_signed) SafeMinimum() {
+  return 0;
+}
+
+template <typename O, typename I>
+constexpr RET_TYPE(is_integral_unsigned_to_signed) SafeMaximum() {
+  using in_type = typename I::c_type;
+  using out_type = typename O::c_type;
+
+  // Equality is missing because in_type::max() > out_type::max() when types
+  // are of the same width.
+  return static_cast<in_type>(sizeof(in_type) < sizeof(out_type)
+                                  ? std::numeric_limits<in_type>::max()
+                                  : std::numeric_limits<out_type>::max());
+}
+
+template <typename O, typename I>
+constexpr RET_TYPE(is_integral_signed_to_unsigned) SafeMinimum() {
+  return 0;
+}
+
+template <typename O, typename I>
+constexpr RET_TYPE(is_integral_signed_to_unsigned) SafeMaximum() {
+  using in_type = typename I::c_type;
+  using out_type = typename O::c_type;
+
+  return static_cast<in_type>(sizeof(in_type) <= sizeof(out_type)
+                                  ? std::numeric_limits<in_type>::max()
+                                  : std::numeric_limits<out_type>::max());
+}
+
+#undef RET_TYPE
+
+template <typename O, typename I>
+struct CastFunctor<
+    O, I,
+    typename std::enable_if<is_number_downcast<O, I>::value ||
+                            is_integral_signed_to_unsigned<O, I>::value ||
+                            is_integral_unsigned_to_signed<O, I>::value>::type> {
   void operator()(FunctionContext* ctx, const CastOptions& options,
                   const ArrayData& input, ArrayData* output) {
     using in_type = typename I::c_type;
@@ -236,8 +324,8 @@ struct CastFunctor<O, I,
     auto out_data = output->GetMutableValues<out_type>(1);
 
     if (!options.allow_int_overflow) {
-      constexpr in_type kMax = static_cast<in_type>(std::numeric_limits<out_type>::max());
-      constexpr in_type kMin = static_cast<in_type>(std::numeric_limits<out_type>::min());
+      constexpr in_type kMax = SafeMaximum<O, I>();
+      constexpr in_type kMin = SafeMinimum<O, I>();
 
       // Null count may be -1 if the input array had been sliced
       if (input.null_count != 0) {
@@ -267,6 +355,22 @@ struct CastFunctor<O, I,
   }
 };
 
+// Float to Integer or Integer to Float
+template <typename O, typename I, typename Enable = void>
+struct is_float_truncate {
+  static constexpr bool value = false;
+};
+
+template <typename O, typename I>
+struct is_float_truncate<
+    O, I,
+    typename std::enable_if<(std::is_base_of<Integer, O>::value &&
+                             std::is_base_of<FloatingPoint, I>::value) ||
+                            (std::is_base_of<Integer, I>::value &&
+                             std::is_base_of<FloatingPoint, O>::value)>::type> {
+  static constexpr bool value = true;
+};
+
 template <typename O, typename I>
 struct CastFunctor<O, I, typename std::enable_if<is_float_truncate<O, I>::value>::type> {
   void operator()(FunctionContext* ctx, const CastOptions& options,
@@ -290,7 +394,8 @@ struct CastFunctor<O, I, typename std::enable_if<is_float_truncate<O, I>::value>
                                                input.length);
         for (int64_t i = 0; i < input.length; ++i) {
           auto out_value = static_cast<out_type>(*in_data);
-          if (ARROW_PREDICT_FALSE(static_cast<in_type>(out_value) != *in_data)) {
+          if (ARROW_PREDICT_FALSE(is_valid_reader.IsSet() &&
+                                  static_cast<in_type>(out_value) != *in_data)) {
             ctx->SetStatus(Status::Invalid("Floating point value truncated"));
           }
           *out_data++ = out_value;
@@ -311,11 +416,31 @@ struct CastFunctor<O, I, typename std::enable_if<is_float_truncate<O, I>::value>
   }
 };
 
+// Leftover of Number combinations that are safe to cast.
+template <typename O, typename I, typename Enable = void>
+struct is_safe_numeric_cast {
+  static constexpr bool value = false;
+};
+
+template <typename O, typename I>
+struct is_safe_numeric_cast<
+    O, I,
+    typename std::enable_if<std::is_base_of<Number, O>::value &&
+                            std::is_base_of<Number, I>::value>::type> {
+  using O_T = typename O::c_type;
+  using I_T = typename I::c_type;
+
+  static constexpr bool value =
+      (std::is_signed<O_T>::value == std::is_signed<I_T>::value) &&
+      (std::is_integral<O_T>::value == std::is_integral<I_T>::value) &&
+      (sizeof(O_T) >= sizeof(I_T)) && (!std::is_same<O, I>::value);
+};
+
 template <typename O, typename I>
 struct CastFunctor<O, I,
-                   typename std::enable_if<is_numeric_cast<O, I>::value &&
+                   typename std::enable_if<is_safe_numeric_cast<O, I>::value &&
                                            !is_float_truncate<O, I>::value &&
-                                           !is_integer_downcast<O, I>::value>::type> {
+                                           !is_number_downcast<O, I>::value>::type> {
   void operator()(FunctionContext* ctx, const CastOptions& options,
                   const ArrayData& input, ArrayData* output) {
     using in_type = typename I::c_type;
@@ -324,6 +449,8 @@ struct CastFunctor<O, I,
     const in_type* in_data = input.GetValues<in_type>(1);
     auto out_data = output->GetMutableValues<out_type>(1);
     for (int64_t i = 0; i < input.length; ++i) {
+      // Due to various checks done via type-trait, the cast is safe and bear
+      // no truncation.
       *out_data++ = static_cast<out_type>(*in_data++);
     }
   }