You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by "pitrou (via GitHub)" <gi...@apache.org> on 2023/06/15 13:20:48 UTC

[GitHub] [arrow] pitrou commented on a diff in pull request #36073: GH-36036: [C++][Parquet] Implement Float16 logical type

pitrou commented on code in PR #36073:
URL: https://github.com/apache/arrow/pull/36073#discussion_r1230932433


##########
cpp/src/parquet/float_internal.h:
##########
@@ -0,0 +1,61 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#pragma once
+
+#include <cstdint>
+#include <cstring>
+
+#include "arrow/util/bit_util.h"
+#include "arrow/util/ubsan.h"
+#include "parquet/types.h"
+
+namespace parquet {
+
+struct float16 {
+  constexpr static uint16_t min() { return 0b1111101111111111; }
+  constexpr static uint16_t max() { return 0b0111101111111111; }
+  constexpr static uint16_t positive_zero() { return 0b0000000000000000; }
+  constexpr static uint16_t negative_zero() { return 0b1000000000000000; }
+
+  static uint8_t* min_ptr() { return min_; }
+  static uint8_t* max_ptr() { return max_; }
+  static uint8_t* positive_zero_ptr() { return positive_zero_; }
+  static uint8_t* negative_zero_ptr() { return negative_zero_; }
+
+  static bool is_nan(uint16_t n) { return (n & 0x7c00) == 0x7c00 && (n & 0x03ff) != 0; }
+  static bool is_zero(uint16_t n) { return (n & 0x7fff) == 0; }
+  static bool signbit(uint16_t n) { return (n & 0x8000) != 0; }
+
+  static uint16_t Pack(const uint8_t* src) {

Review Comment:
   I'm not sure, but the terminology seems a bit confusing to me. For example, in the Python stdlib [pack](https://docs.python.org/3/library/struct.html#struct.pack) means serialize to bytes and [unpack](https://docs.python.org/3/library/struct.html#struct.unpack) means the reverse...



##########
cpp/src/parquet/float_internal.h:
##########
@@ -0,0 +1,61 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#pragma once
+
+#include <cstdint>
+#include <cstring>
+
+#include "arrow/util/bit_util.h"
+#include "arrow/util/ubsan.h"
+#include "parquet/types.h"
+
+namespace parquet {
+
+struct float16 {

Review Comment:
   Do we want to put these helpers in `arrow/util` instead? At some point we'll probably want to process float16 data in Arrow C++ as well...
   



##########
cpp/src/parquet/statistics_test.cc:
##########
@@ -907,6 +921,36 @@ void TestStatisticsSortOrder<FLBAType>::SetValues() {
       .set_max(std::string(reinterpret_cast<const char*>(&vals[8][0]), FLBA_LENGTH));
 }
 
+template <>
+void TestStatisticsSortOrder<Float16LogicalType>::AddNodes(std::string name) {
+  auto node =
+      schema::PrimitiveNode::Make(name, Repetition::REQUIRED, LogicalType::Float16(),
+                                  Type::FIXED_LEN_BYTE_ARRAY, sizeof(uint16_t));
+  fields_.push_back(std::move(node));
+}
+
+template <>
+void TestStatisticsSortOrder<Float16LogicalType>::SetValues() {
+  constexpr int kValueLen = 2;
+  constexpr int kNumBytes = NUM_VALUES * kValueLen;
+
+  const uint16_t packed_vals[NUM_VALUES] = {

Review Comment:
   Add a comment listing the actual float values?



##########
cpp/src/parquet/statistics_test.cc:
##########
@@ -1073,50 +1125,217 @@ void CheckExtrema() {
 TEST(TestStatistic, Int32Extrema) { CheckExtrema<Int32Type>(); }
 TEST(TestStatistic, Int64Extrema) { CheckExtrema<Int64Type>(); }
 
-// PARQUET-1225: Float NaN values may lead to incorrect min-max
-template <typename ParquetType>
-void CheckNaNs() {
-  using T = typename ParquetType::c_type;
+template <typename T>
+class TestFloatStatistics : public ::testing::Test {
+ public:
+  using ParquetType = typename RebindLogical<T>::ParquetType;
+  using c_type = typename ParquetType::c_type;
+
+  void Init();
+  void SetUp() override { this->Init(); }
+
+  bool signbit(c_type val);
+  void CheckEq(const c_type& l, const c_type& r);
+  NodePtr MakeNode(const std::string& name, Repetition::type rep);
+
+  template <typename Stats, typename Values>
+  void CheckMinMaxZeroesSign(Stats stats, const Values& values) {
+    stats->Update(values.data(), values.size(), 0);
+    ASSERT_TRUE(stats->HasMinMax());
+
+    this->CheckEq(stats->min(), positive_zero_);
+    ASSERT_TRUE(this->signbit(stats->min()));
+
+    this->CheckEq(stats->max(), positive_zero_);
+    ASSERT_FALSE(this->signbit(stats->max()));
+  }
+
+  // ARROW-5562: Ensure that -0.0f and 0.0f values are properly handled like in
+  // parquet-mr
+  void TestNegativeZeroes() {
+    NodePtr node = this->MakeNode("f", Repetition::OPTIONAL);
+    ColumnDescriptor descr(node, 1, 1);
+
+    {
+      std::array<c_type, 2> values{negative_zero_, positive_zero_};
+      auto stats = MakeStatistics<ParquetType>(&descr);
+      CheckMinMaxZeroesSign(stats, values);
+    }
+
+    {
+      std::array<c_type, 2> values{positive_zero_, negative_zero_};
+      auto stats = MakeStatistics<ParquetType>(&descr);
+      CheckMinMaxZeroesSign(stats, values);
+    }
+
+    {
+      std::array<c_type, 2> values{negative_zero_, negative_zero_};
+      auto stats = MakeStatistics<ParquetType>(&descr);
+      CheckMinMaxZeroesSign(stats, values);
+    }
+
+    {
+      std::array<c_type, 2> values{positive_zero_, positive_zero_};
+      auto stats = MakeStatistics<ParquetType>(&descr);
+      CheckMinMaxZeroesSign(stats, values);
+    }
+  }
+
+  // PARQUET-1225: Float NaN values may lead to incorrect min-max
+  template <typename Values>
+  void CheckNaNs(ColumnDescriptor* descr, const Values& all_nans, const Values& some_nans,
+                 const Values& other_nans, c_type min, c_type max, uint8_t valid_bitmap,
+                 uint8_t valid_bitmap_no_nans) {
+    auto some_nan_stats = MakeStatistics<ParquetType>(descr);
+    // Ingesting only nans should not yield valid min max
+    AssertUnsetMinMax(some_nan_stats, all_nans);
+    // Ingesting a mix of NaNs and non-NaNs should not yield valid min max.

Review Comment:
   The comment doesn't match the assertion below, does it?



##########
cpp/src/parquet/statistics_test.cc:
##########
@@ -972,12 +1016,20 @@ TEST_F(TestStatisticsSortOrderFLBA, UnknownSortOrder) {
   ASSERT_FALSE(cc_metadata->is_stats_set());
 }
 
+template <typename T>
+static std::string EncodeValue(const T& val) {
+  return std::string(reinterpret_cast<const char*>(&val), sizeof(val));
+}
+static std::string EncodeValue(const FLBA& val, int length = sizeof(uint16_t)) {
+  return std::string(reinterpret_cast<const char*>(val.ptr), length);
+}
+
 template <typename Stats, typename Array, typename T = typename Array::value_type>
 void AssertMinMaxAre(Stats stats, const Array& values, T expected_min, T expected_max) {
   stats->Update(values.data(), values.size(), 0);
   ASSERT_TRUE(stats->HasMinMax());
-  EXPECT_EQ(stats->min(), expected_min);
-  EXPECT_EQ(stats->max(), expected_max);
+  EXPECT_EQ(stats->EncodeMin(), EncodeValue(expected_min));
+  EXPECT_EQ(stats->EncodeMax(), EncodeValue(expected_max));

Review Comment:
   This is because `FLBA` doesn't have a `operator==`, right?



##########
cpp/src/parquet/statistics_test.cc:
##########
@@ -1073,50 +1125,217 @@ void CheckExtrema() {
 TEST(TestStatistic, Int32Extrema) { CheckExtrema<Int32Type>(); }
 TEST(TestStatistic, Int64Extrema) { CheckExtrema<Int64Type>(); }
 
-// PARQUET-1225: Float NaN values may lead to incorrect min-max
-template <typename ParquetType>
-void CheckNaNs() {
-  using T = typename ParquetType::c_type;
+template <typename T>
+class TestFloatStatistics : public ::testing::Test {
+ public:
+  using ParquetType = typename RebindLogical<T>::ParquetType;
+  using c_type = typename ParquetType::c_type;
+
+  void Init();
+  void SetUp() override { this->Init(); }
+
+  bool signbit(c_type val);
+  void CheckEq(const c_type& l, const c_type& r);
+  NodePtr MakeNode(const std::string& name, Repetition::type rep);
+
+  template <typename Stats, typename Values>
+  void CheckMinMaxZeroesSign(Stats stats, const Values& values) {
+    stats->Update(values.data(), values.size(), 0);
+    ASSERT_TRUE(stats->HasMinMax());
+
+    this->CheckEq(stats->min(), positive_zero_);
+    ASSERT_TRUE(this->signbit(stats->min()));
+
+    this->CheckEq(stats->max(), positive_zero_);
+    ASSERT_FALSE(this->signbit(stats->max()));
+  }
+
+  // ARROW-5562: Ensure that -0.0f and 0.0f values are properly handled like in
+  // parquet-mr
+  void TestNegativeZeroes() {
+    NodePtr node = this->MakeNode("f", Repetition::OPTIONAL);
+    ColumnDescriptor descr(node, 1, 1);
+
+    {
+      std::array<c_type, 2> values{negative_zero_, positive_zero_};
+      auto stats = MakeStatistics<ParquetType>(&descr);
+      CheckMinMaxZeroesSign(stats, values);
+    }
+
+    {
+      std::array<c_type, 2> values{positive_zero_, negative_zero_};
+      auto stats = MakeStatistics<ParquetType>(&descr);
+      CheckMinMaxZeroesSign(stats, values);
+    }
+
+    {
+      std::array<c_type, 2> values{negative_zero_, negative_zero_};
+      auto stats = MakeStatistics<ParquetType>(&descr);
+      CheckMinMaxZeroesSign(stats, values);
+    }
+
+    {
+      std::array<c_type, 2> values{positive_zero_, positive_zero_};
+      auto stats = MakeStatistics<ParquetType>(&descr);
+      CheckMinMaxZeroesSign(stats, values);
+    }
+  }
+
+  // PARQUET-1225: Float NaN values may lead to incorrect min-max
+  template <typename Values>
+  void CheckNaNs(ColumnDescriptor* descr, const Values& all_nans, const Values& some_nans,
+                 const Values& other_nans, c_type min, c_type max, uint8_t valid_bitmap,
+                 uint8_t valid_bitmap_no_nans) {
+    auto some_nan_stats = MakeStatistics<ParquetType>(descr);
+    // Ingesting only nans should not yield valid min max
+    AssertUnsetMinMax(some_nan_stats, all_nans);
+    // Ingesting a mix of NaNs and non-NaNs should not yield valid min max.
+    AssertMinMaxAre(some_nan_stats, some_nans, min, max);
+    // Ingesting only nans after a valid min/max, should have not effect
+    AssertMinMaxAre(some_nan_stats, all_nans, min, max);
+
+    some_nan_stats = MakeStatistics<ParquetType>(descr);
+    AssertUnsetMinMax(some_nan_stats, all_nans, &valid_bitmap);
+    // NaNs should not pollute min max when excluded via null bitmap.
+    AssertMinMaxAre(some_nan_stats, some_nans, &valid_bitmap_no_nans, min, max);
+    // Ingesting NaNs with a null bitmap should not change the result.
+    AssertMinMaxAre(some_nan_stats, some_nans, &valid_bitmap, min, max);
+
+    // An array that doesn't start with NaN
+    auto other_stats = MakeStatistics<ParquetType>(descr);
+    AssertMinMaxAre(other_stats, other_nans, min, max);
+  }
+
+  void TestNaNs();
+
+ protected:
+  std::vector<uint8_t> data_buf_;
+  c_type positive_zero_;
+  c_type negative_zero_;
+};
+
+template <typename T>
+void TestFloatStatistics<T>::Init() {
+  positive_zero_ = c_type{};
+  negative_zero_ = -positive_zero_;
+}
+template <>
+void TestFloatStatistics<Float16LogicalType>::Init() {
+  positive_zero_ = c_type{float16::positive_zero_ptr()};
+  negative_zero_ = c_type{float16::negative_zero_ptr()};
+}
+
+template <typename T>
+NodePtr TestFloatStatistics<T>::MakeNode(const std::string& name, Repetition::type rep) {
+  return PrimitiveNode::Make(name, rep, ParquetType::type_num);
+}
+template <>
+NodePtr TestFloatStatistics<Float16LogicalType>::MakeNode(const std::string& name,
+                                                          Repetition::type rep) {
+  return PrimitiveNode::Make(name, rep, LogicalType::Float16(),
+                             Type::FIXED_LEN_BYTE_ARRAY, 2);
+}
 
+template <typename T>
+void TestFloatStatistics<T>::CheckEq(const c_type& l, const c_type& r) {
+  ASSERT_EQ(l, r);
+}
+template <>
+void TestFloatStatistics<Float16LogicalType>::CheckEq(const c_type& a, const c_type& b) {
+  auto l = float16::Pack(a);
+  auto r = float16::Pack(b);
+  if (float16::is_zero(l) && float16::is_zero(r)) return;
+  ASSERT_EQ(l, r);
+}
+
+template <typename T>
+bool TestFloatStatistics<T>::signbit(c_type val) {
+  return std::signbit(val);
+}
+template <>
+bool TestFloatStatistics<Float16LogicalType>::signbit(c_type val) {
+  return float16::signbit(float16::Pack(val));
+}
+
+template <typename T>
+void TestFloatStatistics<T>::TestNaNs() {
   constexpr int kNumValues = 8;
-  NodePtr node = PrimitiveNode::Make("f", Repetition::OPTIONAL, ParquetType::type_num);
+  NodePtr node = this->MakeNode("f", Repetition::OPTIONAL);
   ColumnDescriptor descr(node, 1, 1);
 
-  constexpr T nan = std::numeric_limits<T>::quiet_NaN();
-  constexpr T min = -4.0f;
-  constexpr T max = 3.0f;
+  constexpr c_type nan = std::numeric_limits<c_type>::quiet_NaN();
+  constexpr c_type min = -4.0f;
+  constexpr c_type max = 3.0f;
+
+  std::array<c_type, kNumValues> all_nans{nan, nan, nan, nan, nan, nan, nan, nan};
+  std::array<c_type, kNumValues> some_nans{nan, max, -3.0f, -1.0f, nan, 2.0f, min, nan};
+  std::array<c_type, kNumValues> other_nans{1.5f, max, -3.0f, -1.0f, nan, 2.0f, min, nan};
 
-  std::array<T, kNumValues> all_nans{nan, nan, nan, nan, nan, nan, nan, nan};
-  std::array<T, kNumValues> some_nans{nan, max, -3.0f, -1.0f, nan, 2.0f, min, nan};
   uint8_t valid_bitmap = 0x7F;  // 0b01111111
   // NaNs excluded
   uint8_t valid_bitmap_no_nans = 0x6E;  // 0b01101110
 
-  // Test values
-  auto some_nan_stats = MakeStatistics<ParquetType>(&descr);
-  // Ingesting only nans should not yield valid min max
-  AssertUnsetMinMax(some_nan_stats, all_nans);
-  // Ingesting a mix of NaNs and non-NaNs should not yield valid min max.
-  AssertMinMaxAre(some_nan_stats, some_nans, min, max);
-  // Ingesting only nans after a valid min/max, should have not effect
-  AssertMinMaxAre(some_nan_stats, all_nans, min, max);
+  this->CheckNaNs(&descr, all_nans, some_nans, other_nans, min, max, valid_bitmap,
+                  valid_bitmap_no_nans);
+}
+
+template <>
+void TestFloatStatistics<Float16LogicalType>::TestNaNs() {
+  constexpr int kNumValues = 8;
+  constexpr int kValueLen = sizeof(uint16_t);
+
+  NodePtr node = this->MakeNode("f", Repetition::OPTIONAL);
+  ColumnDescriptor descr(node, 1, 1);
+
+  const uint16_t nan_int = 0b1111110010101010;
+  const uint16_t min_int = 0b1010010111000110;

Review Comment:
   Can you comment on FP values for non-NaN constants?



##########
cpp/src/parquet/float_internal.h:
##########
@@ -0,0 +1,61 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#pragma once
+
+#include <cstdint>
+#include <cstring>
+
+#include "arrow/util/bit_util.h"
+#include "arrow/util/ubsan.h"
+#include "parquet/types.h"
+
+namespace parquet {
+
+struct float16 {
+  constexpr static uint16_t min() { return 0b1111101111111111; }
+  constexpr static uint16_t max() { return 0b0111101111111111; }
+  constexpr static uint16_t positive_zero() { return 0b0000000000000000; }
+  constexpr static uint16_t negative_zero() { return 0b1000000000000000; }
+
+  static uint8_t* min_ptr() { return min_; }
+  static uint8_t* max_ptr() { return max_; }
+  static uint8_t* positive_zero_ptr() { return positive_zero_; }
+  static uint8_t* negative_zero_ptr() { return negative_zero_; }
+
+  static bool is_nan(uint16_t n) { return (n & 0x7c00) == 0x7c00 && (n & 0x03ff) != 0; }
+  static bool is_zero(uint16_t n) { return (n & 0x7fff) == 0; }
+  static bool signbit(uint16_t n) { return (n & 0x8000) != 0; }

Review Comment:
   Why not make all these methods constexpr?



##########
cpp/src/parquet/statistics.cc:
##########
@@ -277,11 +278,54 @@ template <bool is_signed>
 struct CompareHelper<FLBAType, is_signed>
     : public BinaryLikeCompareHelperBase<FLBAType, is_signed> {};
 
+struct Float16CompareHelper {
+  using T = FLBA;
+
+  static T DefaultMin() { return T{float16::max_ptr()}; }
+  static T DefaultMax() { return T{float16::min_ptr()}; }
+
+  static T Coalesce(T val, T fallback) {
+    return val.ptr != nullptr && float16::is_nan(float16::Pack(val)) ? fallback : val;
+  }
+
+  static inline bool Compare(int type_length, const T& a, const T& b) {
+    uint16_t l = float16::Pack(a);
+    uint16_t r = float16::Pack(b);

Review Comment:
   Do we know for sure that neither `l` nor `r` can be NaN here? If so, add `DCHECK`s. Otherwise, false should always be returned if either is NaN (or is it implicit below?).



##########
cpp/src/parquet/float_internal.h:
##########
@@ -0,0 +1,61 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#pragma once
+
+#include <cstdint>
+#include <cstring>
+
+#include "arrow/util/bit_util.h"
+#include "arrow/util/ubsan.h"
+#include "parquet/types.h"
+
+namespace parquet {
+
+struct float16 {
+  constexpr static uint16_t min() { return 0b1111101111111111; }
+  constexpr static uint16_t max() { return 0b0111101111111111; }
+  constexpr static uint16_t positive_zero() { return 0b0000000000000000; }
+  constexpr static uint16_t negative_zero() { return 0b1000000000000000; }
+
+  static uint8_t* min_ptr() { return min_; }
+  static uint8_t* max_ptr() { return max_; }
+  static uint8_t* positive_zero_ptr() { return positive_zero_; }
+  static uint8_t* negative_zero_ptr() { return negative_zero_; }
+
+  static bool is_nan(uint16_t n) { return (n & 0x7c00) == 0x7c00 && (n & 0x03ff) != 0; }
+  static bool is_zero(uint16_t n) { return (n & 0x7fff) == 0; }
+  static bool signbit(uint16_t n) { return (n & 0x8000) != 0; }
+
+  static uint16_t Pack(const uint8_t* src) {
+    return ::arrow::bit_util::FromLittleEndian(::arrow::util::SafeLoadAs<uint16_t>(src));
+  }
+  static uint16_t Pack(const FLBA& src) { return Pack(src.ptr); }
+
+  static uint8_t* Unpack(uint16_t src, uint8_t* dest) {
+    src = ::arrow::bit_util::ToLittleEndian(src);
+    return static_cast<uint8_t*>(std::memcpy(dest, &src, sizeof(src)));

Review Comment:
   Is it worth actually returning the destination pointer?



##########
cpp/src/parquet/statistics.cc:
##########
@@ -277,11 +278,54 @@ template <bool is_signed>
 struct CompareHelper<FLBAType, is_signed>
     : public BinaryLikeCompareHelperBase<FLBAType, is_signed> {};
 
+struct Float16CompareHelper {
+  using T = FLBA;
+
+  static T DefaultMin() { return T{float16::max_ptr()}; }
+  static T DefaultMax() { return T{float16::min_ptr()}; }
+
+  static T Coalesce(T val, T fallback) {
+    return val.ptr != nullptr && float16::is_nan(float16::Pack(val)) ? fallback : val;
+  }
+
+  static inline bool Compare(int type_length, const T& a, const T& b) {
+    uint16_t l = float16::Pack(a);
+    uint16_t r = float16::Pack(b);
+
+    if (l & 0x8000) {

Review Comment:
   Why not use the helper functions, e.g. `signbit`, instead of resorting to magic bitmasks?



##########
cpp/src/parquet/statistics_test.cc:
##########
@@ -1073,50 +1125,217 @@ void CheckExtrema() {
 TEST(TestStatistic, Int32Extrema) { CheckExtrema<Int32Type>(); }
 TEST(TestStatistic, Int64Extrema) { CheckExtrema<Int64Type>(); }
 
-// PARQUET-1225: Float NaN values may lead to incorrect min-max
-template <typename ParquetType>
-void CheckNaNs() {
-  using T = typename ParquetType::c_type;
+template <typename T>
+class TestFloatStatistics : public ::testing::Test {
+ public:
+  using ParquetType = typename RebindLogical<T>::ParquetType;
+  using c_type = typename ParquetType::c_type;
+
+  void Init();
+  void SetUp() override { this->Init(); }
+
+  bool signbit(c_type val);
+  void CheckEq(const c_type& l, const c_type& r);
+  NodePtr MakeNode(const std::string& name, Repetition::type rep);
+
+  template <typename Stats, typename Values>
+  void CheckMinMaxZeroesSign(Stats stats, const Values& values) {
+    stats->Update(values.data(), values.size(), 0);
+    ASSERT_TRUE(stats->HasMinMax());
+
+    this->CheckEq(stats->min(), positive_zero_);

Review Comment:
   By the way, this is where you could perhaps check the encoded form (as +/-0 are semantically equal, but their encodings are different).



##########
cpp/src/parquet/statistics_test.cc:
##########
@@ -1073,50 +1125,217 @@ void CheckExtrema() {
 TEST(TestStatistic, Int32Extrema) { CheckExtrema<Int32Type>(); }
 TEST(TestStatistic, Int64Extrema) { CheckExtrema<Int64Type>(); }
 
-// PARQUET-1225: Float NaN values may lead to incorrect min-max
-template <typename ParquetType>
-void CheckNaNs() {
-  using T = typename ParquetType::c_type;
+template <typename T>
+class TestFloatStatistics : public ::testing::Test {
+ public:
+  using ParquetType = typename RebindLogical<T>::ParquetType;
+  using c_type = typename ParquetType::c_type;
+
+  void Init();
+  void SetUp() override { this->Init(); }
+
+  bool signbit(c_type val);
+  void CheckEq(const c_type& l, const c_type& r);
+  NodePtr MakeNode(const std::string& name, Repetition::type rep);
+
+  template <typename Stats, typename Values>
+  void CheckMinMaxZeroesSign(Stats stats, const Values& values) {
+    stats->Update(values.data(), values.size(), 0);

Review Comment:
   ```suggestion
       stats->Update(values.data(), values.size(), /*null_count=*/ 0);
   ```



##########
cpp/src/parquet/statistics_test.cc:
##########
@@ -1073,50 +1125,217 @@ void CheckExtrema() {
 TEST(TestStatistic, Int32Extrema) { CheckExtrema<Int32Type>(); }
 TEST(TestStatistic, Int64Extrema) { CheckExtrema<Int64Type>(); }
 
-// PARQUET-1225: Float NaN values may lead to incorrect min-max
-template <typename ParquetType>
-void CheckNaNs() {
-  using T = typename ParquetType::c_type;
+template <typename T>
+class TestFloatStatistics : public ::testing::Test {
+ public:
+  using ParquetType = typename RebindLogical<T>::ParquetType;
+  using c_type = typename ParquetType::c_type;
+
+  void Init();
+  void SetUp() override { this->Init(); }
+
+  bool signbit(c_type val);
+  void CheckEq(const c_type& l, const c_type& r);
+  NodePtr MakeNode(const std::string& name, Repetition::type rep);
+
+  template <typename Stats, typename Values>
+  void CheckMinMaxZeroesSign(Stats stats, const Values& values) {
+    stats->Update(values.data(), values.size(), 0);
+    ASSERT_TRUE(stats->HasMinMax());
+
+    this->CheckEq(stats->min(), positive_zero_);
+    ASSERT_TRUE(this->signbit(stats->min()));
+
+    this->CheckEq(stats->max(), positive_zero_);
+    ASSERT_FALSE(this->signbit(stats->max()));
+  }
+
+  // ARROW-5562: Ensure that -0.0f and 0.0f values are properly handled like in
+  // parquet-mr
+  void TestNegativeZeroes() {
+    NodePtr node = this->MakeNode("f", Repetition::OPTIONAL);
+    ColumnDescriptor descr(node, 1, 1);
+
+    {
+      std::array<c_type, 2> values{negative_zero_, positive_zero_};
+      auto stats = MakeStatistics<ParquetType>(&descr);
+      CheckMinMaxZeroesSign(stats, values);
+    }
+
+    {
+      std::array<c_type, 2> values{positive_zero_, negative_zero_};
+      auto stats = MakeStatistics<ParquetType>(&descr);
+      CheckMinMaxZeroesSign(stats, values);
+    }
+
+    {
+      std::array<c_type, 2> values{negative_zero_, negative_zero_};
+      auto stats = MakeStatistics<ParquetType>(&descr);
+      CheckMinMaxZeroesSign(stats, values);
+    }
+
+    {
+      std::array<c_type, 2> values{positive_zero_, positive_zero_};
+      auto stats = MakeStatistics<ParquetType>(&descr);
+      CheckMinMaxZeroesSign(stats, values);
+    }
+  }
+
+  // PARQUET-1225: Float NaN values may lead to incorrect min-max
+  template <typename Values>
+  void CheckNaNs(ColumnDescriptor* descr, const Values& all_nans, const Values& some_nans,
+                 const Values& other_nans, c_type min, c_type max, uint8_t valid_bitmap,
+                 uint8_t valid_bitmap_no_nans) {
+    auto some_nan_stats = MakeStatistics<ParquetType>(descr);
+    // Ingesting only nans should not yield valid min max
+    AssertUnsetMinMax(some_nan_stats, all_nans);
+    // Ingesting a mix of NaNs and non-NaNs should not yield valid min max.
+    AssertMinMaxAre(some_nan_stats, some_nans, min, max);
+    // Ingesting only nans after a valid min/max, should have not effect
+    AssertMinMaxAre(some_nan_stats, all_nans, min, max);
+
+    some_nan_stats = MakeStatistics<ParquetType>(descr);
+    AssertUnsetMinMax(some_nan_stats, all_nans, &valid_bitmap);
+    // NaNs should not pollute min max when excluded via null bitmap.
+    AssertMinMaxAre(some_nan_stats, some_nans, &valid_bitmap_no_nans, min, max);
+    // Ingesting NaNs with a null bitmap should not change the result.
+    AssertMinMaxAre(some_nan_stats, some_nans, &valid_bitmap, min, max);
+
+    // An array that doesn't start with NaN
+    auto other_stats = MakeStatistics<ParquetType>(descr);
+    AssertMinMaxAre(other_stats, other_nans, min, max);
+  }
+
+  void TestNaNs();
+
+ protected:
+  std::vector<uint8_t> data_buf_;
+  c_type positive_zero_;
+  c_type negative_zero_;
+};
+
+template <typename T>
+void TestFloatStatistics<T>::Init() {
+  positive_zero_ = c_type{};
+  negative_zero_ = -positive_zero_;
+}
+template <>
+void TestFloatStatistics<Float16LogicalType>::Init() {
+  positive_zero_ = c_type{float16::positive_zero_ptr()};
+  negative_zero_ = c_type{float16::negative_zero_ptr()};
+}
+
+template <typename T>
+NodePtr TestFloatStatistics<T>::MakeNode(const std::string& name, Repetition::type rep) {
+  return PrimitiveNode::Make(name, rep, ParquetType::type_num);
+}
+template <>
+NodePtr TestFloatStatistics<Float16LogicalType>::MakeNode(const std::string& name,
+                                                          Repetition::type rep) {
+  return PrimitiveNode::Make(name, rep, LogicalType::Float16(),
+                             Type::FIXED_LEN_BYTE_ARRAY, 2);
+}
 
+template <typename T>
+void TestFloatStatistics<T>::CheckEq(const c_type& l, const c_type& r) {
+  ASSERT_EQ(l, r);
+}
+template <>
+void TestFloatStatistics<Float16LogicalType>::CheckEq(const c_type& a, const c_type& b) {
+  auto l = float16::Pack(a);
+  auto r = float16::Pack(b);
+  if (float16::is_zero(l) && float16::is_zero(r)) return;
+  ASSERT_EQ(l, r);

Review Comment:
   Do we care about NaNs here?



##########
cpp/src/parquet/float_internal.h:
##########
@@ -0,0 +1,61 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#pragma once
+
+#include <cstdint>
+#include <cstring>
+
+#include "arrow/util/bit_util.h"
+#include "arrow/util/ubsan.h"
+#include "parquet/types.h"
+
+namespace parquet {
+
+struct float16 {

Review Comment:
   We might even want to define a `arrow::Float16` struct for easier handling of half-precision floats (together with a bunch of operators, and perhaps overrides for `std::isnan`, `std::numeric_limits` and friends)?
   
   For example:
   ```c++
   struct Float16 {
     uint16_t value;
   
     static Float16 FromBytes(const uint8_t*) { ... }
     static Float16 FromBytes(std::array<uint8_t, 2>) { ... }
     std::array<uint8_t, 2> ToBytes() const { ... }
   
     friend bool operator== ...
   };
   
   static_assert(std::is_trivial_v<Float16>);
   ```
   



-- 
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.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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