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/10/05 18:02:22 UTC

[GitHub] [arrow] Luminarys opened a new pull request #8344: Add BasicDecimal256 Multiplication Support

Luminarys opened a new pull request #8344:
URL: https://github.com/apache/arrow/pull/8344


   


----------------------------------------------------------------
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] Luminarys commented on a change in pull request #8344: Add BasicDecimal256 Multiplication Support (PR for decimal256 branch, not master)

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



##########
File path: cpp/src/arrow/util/basic_decimal.cc
##########
@@ -254,67 +254,125 @@ BasicDecimal128& BasicDecimal128::operator>>=(uint32_t bits) {
 
 namespace {
 
-// TODO: Remove this guard once it's used by BasicDecimal256
-#ifndef ARROW_USE_NATIVE_INT128
-// This method losslessly multiplies x and y into a 128 bit unsigned integer
-// whose high bits will be stored in hi and low bits in lo.
-void ExtendAndMultiplyUint64(uint64_t x, uint64_t y, uint64_t* hi, uint64_t* lo) {
+// Convenience wrapper type over 128 bit unsigned integers. We opt not to
+// replace the uint128_t type in int128_internal.h because it would require
+// significantly more implementation work to be done. This class merely
+// provides the minimum necessary set of functions to perform 128+ bit
+// multiplication operations when there may or may not be native support.
 #ifdef ARROW_USE_NATIVE_INT128
-  const __uint128_t r = static_cast<__uint128_t>(x) * y;
-  *lo = r & kInt64Mask;
-  *hi = r >> 64;
+struct uint128_t {
+  uint128_t() {}
+  uint128_t(uint64_t hi, uint64_t lo) : val_((static_cast<__uint128_t>(hi) << 64) | lo) {}
+  uint128_t(const BasicDecimal128& decimal) {
+    val_ = (static_cast<__uint128_t>(decimal.high_bits()) << 64) | decimal.low_bits();
+  }
+
+  uint64_t hi() { return val_ >> 64; }
+  uint64_t lo() { return val_ & kInt64Mask; }
+
+  uint128_t& operator+=(const uint128_t& other) {
+    val_ += other.val_;
+    return *this;
+  }
+
+  uint128_t& operator*=(const uint128_t& other) {
+    val_ *= other.val_;
+    return *this;
+  }
+
+  __uint128_t val_;
+};
+
 #else
-  // If we can't use a native fallback, perform multiplication
+// Multiply two 64 bit word components into a 128 bit result, with high bits
+// stored in hi and low bits in lo.
+inline void ExtendAndMultiply(uint64_t x, uint64_t y, uint64_t* hi, uint64_t* lo) {
+  // Perform multiplication on two 64 bit words x and y into a 128 bit result
   // by splitting up x and y into 32 bit high/low bit components,
   // allowing us to represent the multiplication as
   // x * y = x_lo * y_lo + x_hi * y_lo * 2^32 + y_hi * x_lo * 2^32
-  // + x_hi * y_hi * 2^64.
+  // + x_hi * y_hi * 2^64
   //
-  // Now, consider the final output as lo_lo || lo_hi || hi_lo || hi_hi.
+  // Now, consider the final output as lo_lo || lo_hi || hi_lo || hi_hi
   // Therefore,
   // lo_lo is (x_lo * y_lo)_lo,
   // lo_hi is ((x_lo * y_lo)_hi + (x_hi * y_lo)_lo + (x_lo * y_hi)_lo)_lo,
   // hi_lo is ((x_hi * y_hi)_lo + (x_hi * y_lo)_hi + (x_lo * y_hi)_hi)_hi,
   // hi_hi is (x_hi * y_hi)_hi
-  const uint64_t x_lo = x & kIntMask;
-  const uint64_t y_lo = y & kIntMask;
+  const uint64_t x_lo = x & kInt32Mask;
+  const uint64_t y_lo = y & kInt32Mask;
   const uint64_t x_hi = x >> 32;
   const uint64_t y_hi = y >> 32;
 
   const uint64_t t = x_lo * y_lo;
-  const uint64_t t_lo = t & kIntMask;
+  const uint64_t t_lo = t & kInt32Mask;
   const uint64_t t_hi = t >> 32;
 
   const uint64_t u = x_hi * y_lo + t_hi;
-  const uint64_t u_lo = u & kIntMask;
+  const uint64_t u_lo = u & kInt32Mask;
   const uint64_t u_hi = u >> 32;
 
   const uint64_t v = x_lo * y_hi + u_lo;
   const uint64_t v_hi = v >> 32;
 
   *hi = x_hi * y_hi + u_hi + v_hi;
-  *lo = (v << 32) | t_lo;
-#endif
+  *lo = (v << 32) + t_lo;
 }
-#endif
 
-void MultiplyUint128(uint64_t x_hi, uint64_t x_lo, uint64_t y_hi, uint64_t y_lo,
-                     uint64_t* hi, uint64_t* lo) {
-#ifdef ARROW_USE_NATIVE_INT128
-  const __uint128_t x = (static_cast<__uint128_t>(x_hi) << 64) | x_lo;
-  const __uint128_t y = (static_cast<__uint128_t>(y_hi) << 64) | y_lo;
-  const __uint128_t r = x * y;
-  *lo = r & kInt64Mask;
-  *hi = r >> 64;
-#else
-  // To perform 128 bit multiplication without a native fallback
-  // we first perform lossless 64 bit multiplication of the low
-  // bits, and then add x_hi * y_lo and x_lo * y_hi to the high
-  // bits. Note that we can skip adding x_hi * y_hi because it
-  // always will be over 128 bits.
-  ExtendAndMultiplyUint64(x_lo, y_lo, hi, lo);
-  *hi += (x_hi * y_lo) + (x_lo * y_hi);
+struct uint128_t {
+  uint128_t() {}
+  uint128_t(uint64_t hi, uint64_t lo) : hi_(hi), lo_(lo) {}
+  uint128_t(const BasicDecimal128& decimal) {
+    hi_ = decimal.high_bits();
+    lo_ = decimal.low_bits();
+  }
+
+  uint64_t hi() const { return hi_; }
+  uint64_t lo() const { return lo_; }
+
+  uint128_t& operator+=(const uint128_t& other) {
+    // To deduce the carry bit, we perform "65 bit" addition on the low bits and
+    // seeing if the resulting high bit is 1. This is accomplished by shifting the
+    // low bits to the right by 1 (chopping off the lowest bit), then adding 1 if the
+    // result of adding the two chopped bits would have produced a carry.
+    uint64_t carry = (((lo_ & other.lo_) & 1) + (lo_ >> 1) + (other.lo_ >> 1)) >> 63;
+    hi_ += other.hi_ + carry;
+    lo_ += other.lo_;
+    return *this;
+  }
+
+  uint128_t& operator*=(const uint128_t& other) {
+    uint128_t r;
+    ExtendAndMultiply(lo_, other.lo_, &r.hi_, &r.lo_);
+    r.hi_ += (hi_ * other.lo_) + (lo_ * other.hi_);
+    *this = r;
+    return *this;
+  }
+
+  uint64_t hi_;
+  uint64_t lo_;
+};
 #endif
+
+// Multiplies two N * 64 bit unsigned integer types, represented by a uint64_t
+// array into a same sized output. Elements in the array should be in
+// little endian order, and output will be the same. Overflow in multiplication
+// is considered undefined behavior and will not be reported.

Review comment:
       When I say undefined here, I mean the value should not be relied on and is an implementation detail, i.e. people should only be calling this if they know the result will not overflow or do not care what happens if it does. Undefined Behavior maybe isn't correct because it implies the same kind of UB you get when you dereference a nullptr, etc.
   
   I've tweaked the documentation though to reflect what actually happens since this file is the only consumer of the function anyways.




----------------------------------------------------------------
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] Luminarys commented on a change in pull request #8344: Add BasicDecimal256 Multiplication Support (PR for decimal256 branch, not master)

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



##########
File path: cpp/src/arrow/util/basic_decimal.cc
##########
@@ -254,69 +252,148 @@ BasicDecimal128& BasicDecimal128::operator>>=(uint32_t bits) {
 
 namespace {
 
-// TODO: Remove this guard once it's used by BasicDecimal256
-#ifndef ARROW_USE_NATIVE_INT128
-// This method losslessly multiplies x and y into a 128 bit unsigned integer
-// whose high bits will be stored in hi and low bits in lo.
-void ExtendAndMultiplyUint64(uint64_t x, uint64_t y, uint64_t* hi, uint64_t* lo) {
-#ifdef ARROW_USE_NATIVE_INT128
-  const __uint128_t r = static_cast<__uint128_t>(x) * y;
-  *lo = r & kInt64Mask;
-  *hi = r >> 64;
-#else
-  // If we can't use a native fallback, perform multiplication
-  // by splitting up x and y into 32 bit high/low bit components,
+// Multiply two N bit word components into a 2*N bit result, with high bits
+// stored in hi and low bits in lo.
+template <typename Word>
+void ExtendAndMultiplyUint(Word x, Word y, Word* hi, Word* lo) {

Review comment:
       I realized that I wasn't using the native path prior, which is why the benchmark was so slow. Updated, new results are 32 ns when __uint128_t is used and 65 ns when uint64_t is used, which I think is more reasonable.

##########
File path: cpp/src/arrow/util/basic_decimal.cc
##########
@@ -254,69 +252,148 @@ BasicDecimal128& BasicDecimal128::operator>>=(uint32_t bits) {
 
 namespace {
 
-// TODO: Remove this guard once it's used by BasicDecimal256
-#ifndef ARROW_USE_NATIVE_INT128
-// This method losslessly multiplies x and y into a 128 bit unsigned integer
-// whose high bits will be stored in hi and low bits in lo.
-void ExtendAndMultiplyUint64(uint64_t x, uint64_t y, uint64_t* hi, uint64_t* lo) {
-#ifdef ARROW_USE_NATIVE_INT128
-  const __uint128_t r = static_cast<__uint128_t>(x) * y;
-  *lo = r & kInt64Mask;
-  *hi = r >> 64;
-#else
-  // If we can't use a native fallback, perform multiplication
-  // by splitting up x and y into 32 bit high/low bit components,
+// Multiply two N bit word components into a 2*N bit result, with high bits
+// stored in hi and low bits in lo.
+template <typename Word>
+void ExtendAndMultiplyUint(Word x, Word y, Word* hi, Word* lo) {
+  // Perform multiplication on two N bit words x and y into a 2*N bit result
+  // by splitting up x and y into N/2 bit high/low bit components,
   // allowing us to represent the multiplication as
-  // x * y = x_lo * y_lo + x_hi * y_lo * 2^32 + y_hi * x_lo * 2^32
-  // + x_hi * y_hi * 2^64.
+  // x * y = x_lo * y_lo + x_hi * y_lo * 2^N/2 + y_hi * x_lo * 2^N/2
+  // + x_hi * y_hi * 2^N
   //
-  // Now, consider the final output as lo_lo || lo_hi || hi_lo || hi_hi.
+  // Now, consider the final output as lo_lo || lo_hi || hi_lo || hi_hi
   // Therefore,
   // lo_lo is (x_lo * y_lo)_lo,
   // lo_hi is ((x_lo * y_lo)_hi + (x_hi * y_lo)_lo + (x_lo * y_hi)_lo)_lo,
   // hi_lo is ((x_hi * y_hi)_lo + (x_hi * y_lo)_hi + (x_lo * y_hi)_hi)_hi,
   // hi_hi is (x_hi * y_hi)_hi
-  const uint64_t x_lo = x & kIntMask;
-  const uint64_t y_lo = y & kIntMask;
-  const uint64_t x_hi = x >> 32;
-  const uint64_t y_hi = y >> 32;
+  constexpr Word kHighBitShift = sizeof(Word) * 4;
+  constexpr Word kLowBitMask = (static_cast<Word>(1) << kHighBitShift) - 1;
 
-  const uint64_t t = x_lo * y_lo;
-  const uint64_t t_lo = t & kIntMask;
-  const uint64_t t_hi = t >> 32;
+  const Word x_lo = x & kLowBitMask;
+  const Word y_lo = y & kLowBitMask;
+  const Word x_hi = x >> kHighBitShift;
+  const Word y_hi = y >> kHighBitShift;
 
-  const uint64_t u = x_hi * y_lo + t_hi;
-  const uint64_t u_lo = u & kIntMask;
-  const uint64_t u_hi = u >> 32;
+  const Word t = x_lo * y_lo;
+  const Word t_lo = t & kLowBitMask;
+  const Word t_hi = t >> kHighBitShift;
 
-  const uint64_t v = x_lo * y_hi + u_lo;
-  const uint64_t v_hi = v >> 32;
+  const Word u = x_hi * y_lo + t_hi;
+  const Word u_lo = u & kLowBitMask;
+  const Word u_hi = u >> kHighBitShift;
+
+  const Word v = x_lo * y_hi + u_lo;
+  const Word v_hi = v >> kHighBitShift;
 
   *hi = x_hi * y_hi + u_hi + v_hi;
-  *lo = (v << 32) | t_lo;
-#endif
+  *lo = (v << kHighBitShift) + t_lo;
+}
+
+// Convenience wrapper type over 128 bit unsigned integers
+#ifdef ARROW_USE_NATIVE_INT128
+struct uint128_t {

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] Luminarys commented on pull request #8344: Add BasicDecimal256 Multiplication Support (PR for decimal256 branch, not master)

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


   Added benchmark, multiplication takes ~21ns.


----------------------------------------------------------------
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 #8344: Add BasicDecimal256 Multiplication Support (PR for decimal256 branch, not master)

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


   @Luminarys have you looked at the CI errors (I think there might be a few flaky things going on but wanted to check that you were ok merging)?


----------------------------------------------------------------
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] MingyuZhong commented on a change in pull request #8344: Add BasicDecimal256 Multiplication Support (PR for decimal256 branch, not master)

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



##########
File path: cpp/src/arrow/util/basic_decimal.cc
##########
@@ -254,69 +252,148 @@ BasicDecimal128& BasicDecimal128::operator>>=(uint32_t bits) {
 
 namespace {
 
-// TODO: Remove this guard once it's used by BasicDecimal256
-#ifndef ARROW_USE_NATIVE_INT128
-// This method losslessly multiplies x and y into a 128 bit unsigned integer
-// whose high bits will be stored in hi and low bits in lo.
-void ExtendAndMultiplyUint64(uint64_t x, uint64_t y, uint64_t* hi, uint64_t* lo) {
-#ifdef ARROW_USE_NATIVE_INT128
-  const __uint128_t r = static_cast<__uint128_t>(x) * y;
-  *lo = r & kInt64Mask;
-  *hi = r >> 64;
-#else
-  // If we can't use a native fallback, perform multiplication
-  // by splitting up x and y into 32 bit high/low bit components,
+// Multiply two N bit word components into a 2*N bit result, with high bits
+// stored in hi and low bits in lo.
+template <typename Word>
+void ExtendAndMultiplyUint(Word x, Word y, Word* hi, Word* lo) {
+  // Perform multiplication on two N bit words x and y into a 2*N bit result
+  // by splitting up x and y into N/2 bit high/low bit components,
   // allowing us to represent the multiplication as
-  // x * y = x_lo * y_lo + x_hi * y_lo * 2^32 + y_hi * x_lo * 2^32
-  // + x_hi * y_hi * 2^64.
+  // x * y = x_lo * y_lo + x_hi * y_lo * 2^N/2 + y_hi * x_lo * 2^N/2
+  // + x_hi * y_hi * 2^N
   //
-  // Now, consider the final output as lo_lo || lo_hi || hi_lo || hi_hi.
+  // Now, consider the final output as lo_lo || lo_hi || hi_lo || hi_hi
   // Therefore,
   // lo_lo is (x_lo * y_lo)_lo,
   // lo_hi is ((x_lo * y_lo)_hi + (x_hi * y_lo)_lo + (x_lo * y_hi)_lo)_lo,
   // hi_lo is ((x_hi * y_hi)_lo + (x_hi * y_lo)_hi + (x_lo * y_hi)_hi)_hi,
   // hi_hi is (x_hi * y_hi)_hi
-  const uint64_t x_lo = x & kIntMask;
-  const uint64_t y_lo = y & kIntMask;
-  const uint64_t x_hi = x >> 32;
-  const uint64_t y_hi = y >> 32;
+  constexpr Word kHighBitShift = sizeof(Word) * 4;
+  constexpr Word kLowBitMask = (static_cast<Word>(1) << kHighBitShift) - 1;
 
-  const uint64_t t = x_lo * y_lo;
-  const uint64_t t_lo = t & kIntMask;
-  const uint64_t t_hi = t >> 32;
+  const Word x_lo = x & kLowBitMask;
+  const Word y_lo = y & kLowBitMask;
+  const Word x_hi = x >> kHighBitShift;
+  const Word y_hi = y >> kHighBitShift;
 
-  const uint64_t u = x_hi * y_lo + t_hi;
-  const uint64_t u_lo = u & kIntMask;
-  const uint64_t u_hi = u >> 32;
+  const Word t = x_lo * y_lo;
+  const Word t_lo = t & kLowBitMask;
+  const Word t_hi = t >> kHighBitShift;
 
-  const uint64_t v = x_lo * y_hi + u_lo;
-  const uint64_t v_hi = v >> 32;
+  const Word u = x_hi * y_lo + t_hi;
+  const Word u_lo = u & kLowBitMask;
+  const Word u_hi = u >> kHighBitShift;
+
+  const Word v = x_lo * y_hi + u_lo;
+  const Word v_hi = v >> kHighBitShift;
 
   *hi = x_hi * y_hi + u_hi + v_hi;
-  *lo = (v << 32) | t_lo;
-#endif
+  *lo = (v << kHighBitShift) + t_lo;
+}
+
+// Convenience wrapper type over 128 bit unsigned integers
+#ifdef ARROW_USE_NATIVE_INT128
+struct uint128_t {

Review comment:
       I think it's better to change this struct into a class, move it to int128_internal.h (replace the existing alias), and add unit tests on the multiplication. The tests should use boost::uint128_t for verification.

##########
File path: cpp/src/arrow/util/basic_decimal.cc
##########
@@ -254,69 +252,148 @@ BasicDecimal128& BasicDecimal128::operator>>=(uint32_t bits) {
 
 namespace {
 
-// TODO: Remove this guard once it's used by BasicDecimal256
-#ifndef ARROW_USE_NATIVE_INT128
-// This method losslessly multiplies x and y into a 128 bit unsigned integer
-// whose high bits will be stored in hi and low bits in lo.
-void ExtendAndMultiplyUint64(uint64_t x, uint64_t y, uint64_t* hi, uint64_t* lo) {
-#ifdef ARROW_USE_NATIVE_INT128
-  const __uint128_t r = static_cast<__uint128_t>(x) * y;
-  *lo = r & kInt64Mask;
-  *hi = r >> 64;
-#else
-  // If we can't use a native fallback, perform multiplication
-  // by splitting up x and y into 32 bit high/low bit components,
+// Multiply two N bit word components into a 2*N bit result, with high bits
+// stored in hi and low bits in lo.
+template <typename Word>
+void ExtendAndMultiplyUint(Word x, Word y, Word* hi, Word* lo) {
+  // Perform multiplication on two N bit words x and y into a 2*N bit result
+  // by splitting up x and y into N/2 bit high/low bit components,
   // allowing us to represent the multiplication as
-  // x * y = x_lo * y_lo + x_hi * y_lo * 2^32 + y_hi * x_lo * 2^32
-  // + x_hi * y_hi * 2^64.
+  // x * y = x_lo * y_lo + x_hi * y_lo * 2^N/2 + y_hi * x_lo * 2^N/2
+  // + x_hi * y_hi * 2^N
   //
-  // Now, consider the final output as lo_lo || lo_hi || hi_lo || hi_hi.
+  // Now, consider the final output as lo_lo || lo_hi || hi_lo || hi_hi
   // Therefore,
   // lo_lo is (x_lo * y_lo)_lo,
   // lo_hi is ((x_lo * y_lo)_hi + (x_hi * y_lo)_lo + (x_lo * y_hi)_lo)_lo,
   // hi_lo is ((x_hi * y_hi)_lo + (x_hi * y_lo)_hi + (x_lo * y_hi)_hi)_hi,
   // hi_hi is (x_hi * y_hi)_hi
-  const uint64_t x_lo = x & kIntMask;
-  const uint64_t y_lo = y & kIntMask;
-  const uint64_t x_hi = x >> 32;
-  const uint64_t y_hi = y >> 32;
+  constexpr Word kHighBitShift = sizeof(Word) * 4;
+  constexpr Word kLowBitMask = (static_cast<Word>(1) << kHighBitShift) - 1;
 
-  const uint64_t t = x_lo * y_lo;
-  const uint64_t t_lo = t & kIntMask;
-  const uint64_t t_hi = t >> 32;
+  const Word x_lo = x & kLowBitMask;
+  const Word y_lo = y & kLowBitMask;
+  const Word x_hi = x >> kHighBitShift;
+  const Word y_hi = y >> kHighBitShift;
 
-  const uint64_t u = x_hi * y_lo + t_hi;
-  const uint64_t u_lo = u & kIntMask;
-  const uint64_t u_hi = u >> 32;
+  const Word t = x_lo * y_lo;
+  const Word t_lo = t & kLowBitMask;
+  const Word t_hi = t >> kHighBitShift;
 
-  const uint64_t v = x_lo * y_hi + u_lo;
-  const uint64_t v_hi = v >> 32;
+  const Word u = x_hi * y_lo + t_hi;
+  const Word u_lo = u & kLowBitMask;
+  const Word u_hi = u >> kHighBitShift;
+
+  const Word v = x_lo * y_hi + u_lo;
+  const Word v_hi = v >> kHighBitShift;
 
   *hi = x_hi * y_hi + u_hi + v_hi;
-  *lo = (v << 32) | t_lo;
-#endif
+  *lo = (v << kHighBitShift) + t_lo;
+}
+
+// Convenience wrapper type over 128 bit unsigned integers
+#ifdef ARROW_USE_NATIVE_INT128
+struct uint128_t {
+  uint128_t() {}
+  uint128_t(uint64_t hi, uint64_t lo) : val_((static_cast<__uint128_t>(hi) << 64) | lo) {}
+  uint128_t(const BasicDecimal128& decimal) {
+    val_ = (static_cast<__uint128_t>(decimal.high_bits()) << 64) | decimal.low_bits();
+  }
+
+  uint64_t hi() { return val_ >> 64; }
+  uint64_t lo() { return val_ & kInt64Mask; }
+
+  uint128_t& operator+=(const uint128_t& other) {
+    val_ += other.val_;
+    return *this;
+  }
+
+  __uint128_t val_;
+};
+
+uint128_t operator*(const uint128_t& left, const uint128_t& right) {
+  uint128_t r;
+  r.val_ = left.val_ * right.val_;
+  return r;
+}
+#else
+struct uint128_t {
+  uint128_t() {}
+  uint128_t(uint64_t hi, uint64_t lo) : hi_(hi), lo_(lo) {}
+  uint128_t(const BasicDecimal128& decimal) {
+    hi_ = decimal.high_bits();
+    lo_ = decimal.low_bits();
+  }
+
+  uint64_t hi() const { return hi_; }
+  uint64_t lo() const { return lo_; }
+
+  uint128_t& operator+=(const uint128_t& other) {
+    // To deduce the carry bit, we perform "65 bit" addition on the low bits and
+    // seeing if the resulting high bit is 1. This is accomplished by shifting the
+    // low bits to the right by 1 (chopping off the lowest bit), then adding 1 if the
+    // result of adding the two chopped bits would have produced a carry.
+    uint64_t carry = (((lo_ & other.lo_) & 1) + (lo_ >> 1) + (other.lo_ >> 1)) >> 63;
+    hi_ += other.hi_ + carry;
+    lo_ += other.lo_;
+    return *this;
+  }
+
+  uint64_t hi_;
+  uint64_t lo_;
+};
+
+uint128_t operator*(const uint128_t& left, const uint128_t& right) {
+  uint128_t r;
+  ExtendAndMultiplyUint(left.lo_, right.lo_, &r.hi_, &r.lo_);
+  r.hi_ += (left.hi_ * right.lo_) + (left.lo_ * right.hi_);
+  return r;
 }
 #endif
 
-void MultiplyUint128(uint64_t x_hi, uint64_t x_lo, uint64_t y_hi, uint64_t y_lo,
-                     uint64_t* hi, uint64_t* lo) {
+void ExtendAndMultiplyUint128(uint128_t x, uint128_t y, uint128_t* hi, uint128_t* lo) {

Review comment:
       This can be a static method of uint128_t.

##########
File path: cpp/src/arrow/util/basic_decimal.cc
##########
@@ -254,69 +252,148 @@ BasicDecimal128& BasicDecimal128::operator>>=(uint32_t bits) {
 
 namespace {
 
-// TODO: Remove this guard once it's used by BasicDecimal256
-#ifndef ARROW_USE_NATIVE_INT128
-// This method losslessly multiplies x and y into a 128 bit unsigned integer
-// whose high bits will be stored in hi and low bits in lo.
-void ExtendAndMultiplyUint64(uint64_t x, uint64_t y, uint64_t* hi, uint64_t* lo) {
-#ifdef ARROW_USE_NATIVE_INT128
-  const __uint128_t r = static_cast<__uint128_t>(x) * y;
-  *lo = r & kInt64Mask;
-  *hi = r >> 64;
-#else
-  // If we can't use a native fallback, perform multiplication
-  // by splitting up x and y into 32 bit high/low bit components,
+// Multiply two N bit word components into a 2*N bit result, with high bits
+// stored in hi and low bits in lo.
+template <typename Word>
+void ExtendAndMultiplyUint(Word x, Word y, Word* hi, Word* lo) {

Review comment:
       I think it's simpler if this method handles only uint64_t, and there is another method that takes std::array<uint64_t, n> and uses for loops like https://github.com/google/zetasql/blob/master/zetasql/common/multiprecision_int.h#L723. This way, ExtendAndMultiplyUint128 doesn't need to repeat the similar pattern.




----------------------------------------------------------------
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] Luminarys edited a comment on pull request #8344: Add BasicDecimal256 Multiplication Support (PR for decimal256 branch, not master)

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


   It turns out one of the check failures is due to a compiler bug in Clang, I've tweaked the definition structure of the BasicDecimal256 header to handle 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] Luminarys commented on a change in pull request #8344: Add BasicDecimal256 Multiplication Support (PR for decimal256 branch, not master)

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



##########
File path: cpp/src/arrow/util/basic_decimal.cc
##########
@@ -254,67 +252,126 @@ BasicDecimal128& BasicDecimal128::operator>>=(uint32_t bits) {
 
 namespace {
 
-// TODO: Remove this guard once it's used by BasicDecimal256
-#ifndef ARROW_USE_NATIVE_INT128
-// This method losslessly multiplies x and y into a 128 bit unsigned integer
-// whose high bits will be stored in hi and low bits in lo.
-void ExtendAndMultiplyUint64(uint64_t x, uint64_t y, uint64_t* hi, uint64_t* lo) {
-#ifdef ARROW_USE_NATIVE_INT128
-  const __uint128_t r = static_cast<__uint128_t>(x) * y;
-  *lo = r & kInt64Mask;
-  *hi = r >> 64;
-#else
-  // If we can't use a native fallback, perform multiplication
-  // by splitting up x and y into 32 bit high/low bit components,
+// Multiply two N bit word components into a 2*N bit result, with high bits
+// stored in hi and low bits in lo.
+template <typename Word>
+inline void ExtendAndMultiplyUint(Word x, Word y, Word* hi, Word* lo) {

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] Luminarys commented on a change in pull request #8344: Add BasicDecimal256 Multiplication Support (PR for decimal256 branch, not master)

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



##########
File path: cpp/src/arrow/util/basic_decimal.cc
##########
@@ -254,67 +252,126 @@ BasicDecimal128& BasicDecimal128::operator>>=(uint32_t bits) {
 
 namespace {
 
-// TODO: Remove this guard once it's used by BasicDecimal256
-#ifndef ARROW_USE_NATIVE_INT128
-// This method losslessly multiplies x and y into a 128 bit unsigned integer
-// whose high bits will be stored in hi and low bits in lo.
-void ExtendAndMultiplyUint64(uint64_t x, uint64_t y, uint64_t* hi, uint64_t* lo) {
-#ifdef ARROW_USE_NATIVE_INT128
-  const __uint128_t r = static_cast<__uint128_t>(x) * y;
-  *lo = r & kInt64Mask;
-  *hi = r >> 64;
-#else
-  // If we can't use a native fallback, perform multiplication
-  // by splitting up x and y into 32 bit high/low bit components,
+// Multiply two N bit word components into a 2*N bit result, with high bits
+// stored in hi and low bits in lo.
+template <typename Word>
+inline void ExtendAndMultiplyUint(Word x, Word y, Word* hi, Word* lo) {
+  // Perform multiplication on two N bit words x and y into a 2*N bit result
+  // by splitting up x and y into N/2 bit high/low bit components,
   // allowing us to represent the multiplication as
-  // x * y = x_lo * y_lo + x_hi * y_lo * 2^32 + y_hi * x_lo * 2^32
-  // + x_hi * y_hi * 2^64.
+  // x * y = x_lo * y_lo + x_hi * y_lo * 2^N/2 + y_hi * x_lo * 2^N/2
+  // + x_hi * y_hi * 2^N
   //
-  // Now, consider the final output as lo_lo || lo_hi || hi_lo || hi_hi.
+  // Now, consider the final output as lo_lo || lo_hi || hi_lo || hi_hi
   // Therefore,
   // lo_lo is (x_lo * y_lo)_lo,
   // lo_hi is ((x_lo * y_lo)_hi + (x_hi * y_lo)_lo + (x_lo * y_hi)_lo)_lo,
   // hi_lo is ((x_hi * y_hi)_lo + (x_hi * y_lo)_hi + (x_lo * y_hi)_hi)_hi,
   // hi_hi is (x_hi * y_hi)_hi
-  const uint64_t x_lo = x & kIntMask;
-  const uint64_t y_lo = y & kIntMask;
-  const uint64_t x_hi = x >> 32;
-  const uint64_t y_hi = y >> 32;
+  constexpr Word kHighBitShift = sizeof(Word) * 4;
+  constexpr Word kLowBitMask = (static_cast<Word>(1) << kHighBitShift) - 1;
 
-  const uint64_t t = x_lo * y_lo;
-  const uint64_t t_lo = t & kIntMask;
-  const uint64_t t_hi = t >> 32;
+  const Word x_lo = x & kLowBitMask;
+  const Word y_lo = y & kLowBitMask;
+  const Word x_hi = x >> kHighBitShift;
+  const Word y_hi = y >> kHighBitShift;
 
-  const uint64_t u = x_hi * y_lo + t_hi;
-  const uint64_t u_lo = u & kIntMask;
-  const uint64_t u_hi = u >> 32;
+  const Word t = x_lo * y_lo;
+  const Word t_lo = t & kLowBitMask;
+  const Word t_hi = t >> kHighBitShift;
 
-  const uint64_t v = x_lo * y_hi + u_lo;
-  const uint64_t v_hi = v >> 32;
+  const Word u = x_hi * y_lo + t_hi;
+  const Word u_lo = u & kLowBitMask;
+  const Word u_hi = u >> kHighBitShift;
+
+  const Word v = x_lo * y_hi + u_lo;
+  const Word v_hi = v >> kHighBitShift;
 
   *hi = x_hi * y_hi + u_hi + v_hi;
-  *lo = (v << 32) | t_lo;
-#endif
+  *lo = (v << kHighBitShift) + t_lo;
 }
-#endif
 
-void MultiplyUint128(uint64_t x_hi, uint64_t x_lo, uint64_t y_hi, uint64_t y_lo,
-                     uint64_t* hi, uint64_t* lo) {
+// Convenience wrapper type over 128 bit unsigned integers. We opt not to
+// replace the uint128_t type in int128_internal.h because it would require
+// significantly more implementation work to be done. This class merely
+// provides the minimum necessary set of functions to perform 128+ bit
+// multiplication operations when there may or may not be native support.
 #ifdef ARROW_USE_NATIVE_INT128
-  const __uint128_t x = (static_cast<__uint128_t>(x_hi) << 64) | x_lo;
-  const __uint128_t y = (static_cast<__uint128_t>(y_hi) << 64) | y_lo;
-  const __uint128_t r = x * y;
-  *lo = r & kInt64Mask;
-  *hi = r >> 64;
+struct uint128_t {
+  uint128_t() {}
+  uint128_t(uint64_t hi, uint64_t lo) : val_((static_cast<__uint128_t>(hi) << 64) | lo) {}
+  uint128_t(const BasicDecimal128& decimal) {
+    val_ = (static_cast<__uint128_t>(decimal.high_bits()) << 64) | decimal.low_bits();
+  }
+
+  uint64_t hi() { return val_ >> 64; }
+  uint64_t lo() { return val_ & kInt64Mask; }
+
+  uint128_t& operator+=(const uint128_t& other) {
+    val_ += other.val_;
+    return *this;
+  }
+
+  __uint128_t val_;
+};
+
+uint128_t operator*(const uint128_t& left, const uint128_t& right) {
+  uint128_t r;
+  r.val_ = left.val_ * right.val_;
+  return r;
+}
 #else
-  // To perform 128 bit multiplication without a native fallback
-  // we first perform lossless 64 bit multiplication of the low
-  // bits, and then add x_hi * y_lo and x_lo * y_hi to the high
-  // bits. Note that we can skip adding x_hi * y_hi because it
-  // always will be over 128 bits.
-  ExtendAndMultiplyUint64(x_lo, y_lo, hi, lo);
-  *hi += (x_hi * y_lo) + (x_lo * y_hi);
+struct uint128_t {
+  uint128_t() {}
+  uint128_t(uint64_t hi, uint64_t lo) : hi_(hi), lo_(lo) {}
+  uint128_t(const BasicDecimal128& decimal) {
+    hi_ = decimal.high_bits();
+    lo_ = decimal.low_bits();
+  }
+
+  uint64_t hi() const { return hi_; }
+  uint64_t lo() const { return lo_; }
+
+  uint128_t& operator+=(const uint128_t& other) {
+    // To deduce the carry bit, we perform "65 bit" addition on the low bits and
+    // seeing if the resulting high bit is 1. This is accomplished by shifting the
+    // low bits to the right by 1 (chopping off the lowest bit), then adding 1 if the
+    // result of adding the two chopped bits would have produced a carry.
+    uint64_t carry = (((lo_ & other.lo_) & 1) + (lo_ >> 1) + (other.lo_ >> 1)) >> 63;
+    hi_ += other.hi_ + carry;
+    lo_ += other.lo_;
+    return *this;
+  }
+
+  uint64_t hi_;
+  uint64_t lo_;
+};
+
+uint128_t operator*(const uint128_t& left, const uint128_t& right) {
+  uint128_t r;
+  ExtendAndMultiplyUint(left.lo_, right.lo_, &r.hi_, &r.lo_);
+  r.hi_ += (left.hi_ * right.lo_) + (left.lo_ * right.hi_);
+  return r;
+}
 #endif
+
+// Multiplies two N * 64 bit unsigned integer types, represented by a uint64_t

Review comment:
       Done.

##########
File path: cpp/src/arrow/util/basic_decimal.cc
##########
@@ -254,67 +252,126 @@ BasicDecimal128& BasicDecimal128::operator>>=(uint32_t bits) {
 
 namespace {
 
-// TODO: Remove this guard once it's used by BasicDecimal256
-#ifndef ARROW_USE_NATIVE_INT128
-// This method losslessly multiplies x and y into a 128 bit unsigned integer
-// whose high bits will be stored in hi and low bits in lo.
-void ExtendAndMultiplyUint64(uint64_t x, uint64_t y, uint64_t* hi, uint64_t* lo) {
-#ifdef ARROW_USE_NATIVE_INT128
-  const __uint128_t r = static_cast<__uint128_t>(x) * y;
-  *lo = r & kInt64Mask;
-  *hi = r >> 64;
-#else
-  // If we can't use a native fallback, perform multiplication
-  // by splitting up x and y into 32 bit high/low bit components,
+// Multiply two N bit word components into a 2*N bit result, with high bits
+// stored in hi and low bits in lo.
+template <typename Word>
+inline void ExtendAndMultiplyUint(Word x, Word y, Word* hi, Word* lo) {
+  // Perform multiplication on two N bit words x and y into a 2*N bit result
+  // by splitting up x and y into N/2 bit high/low bit components,
   // allowing us to represent the multiplication as
-  // x * y = x_lo * y_lo + x_hi * y_lo * 2^32 + y_hi * x_lo * 2^32
-  // + x_hi * y_hi * 2^64.
+  // x * y = x_lo * y_lo + x_hi * y_lo * 2^N/2 + y_hi * x_lo * 2^N/2
+  // + x_hi * y_hi * 2^N
   //
-  // Now, consider the final output as lo_lo || lo_hi || hi_lo || hi_hi.
+  // Now, consider the final output as lo_lo || lo_hi || hi_lo || hi_hi
   // Therefore,
   // lo_lo is (x_lo * y_lo)_lo,
   // lo_hi is ((x_lo * y_lo)_hi + (x_hi * y_lo)_lo + (x_lo * y_hi)_lo)_lo,
   // hi_lo is ((x_hi * y_hi)_lo + (x_hi * y_lo)_hi + (x_lo * y_hi)_hi)_hi,
   // hi_hi is (x_hi * y_hi)_hi
-  const uint64_t x_lo = x & kIntMask;
-  const uint64_t y_lo = y & kIntMask;
-  const uint64_t x_hi = x >> 32;
-  const uint64_t y_hi = y >> 32;
+  constexpr Word kHighBitShift = sizeof(Word) * 4;
+  constexpr Word kLowBitMask = (static_cast<Word>(1) << kHighBitShift) - 1;
 
-  const uint64_t t = x_lo * y_lo;
-  const uint64_t t_lo = t & kIntMask;
-  const uint64_t t_hi = t >> 32;
+  const Word x_lo = x & kLowBitMask;
+  const Word y_lo = y & kLowBitMask;
+  const Word x_hi = x >> kHighBitShift;
+  const Word y_hi = y >> kHighBitShift;
 
-  const uint64_t u = x_hi * y_lo + t_hi;
-  const uint64_t u_lo = u & kIntMask;
-  const uint64_t u_hi = u >> 32;
+  const Word t = x_lo * y_lo;
+  const Word t_lo = t & kLowBitMask;
+  const Word t_hi = t >> kHighBitShift;
 
-  const uint64_t v = x_lo * y_hi + u_lo;
-  const uint64_t v_hi = v >> 32;
+  const Word u = x_hi * y_lo + t_hi;
+  const Word u_lo = u & kLowBitMask;
+  const Word u_hi = u >> kHighBitShift;
+
+  const Word v = x_lo * y_hi + u_lo;
+  const Word v_hi = v >> kHighBitShift;
 
   *hi = x_hi * y_hi + u_hi + v_hi;
-  *lo = (v << 32) | t_lo;
-#endif
+  *lo = (v << kHighBitShift) + t_lo;
 }
-#endif
 
-void MultiplyUint128(uint64_t x_hi, uint64_t x_lo, uint64_t y_hi, uint64_t y_lo,
-                     uint64_t* hi, uint64_t* lo) {
+// Convenience wrapper type over 128 bit unsigned integers. We opt not to
+// replace the uint128_t type in int128_internal.h because it would require
+// significantly more implementation work to be done. This class merely
+// provides the minimum necessary set of functions to perform 128+ bit
+// multiplication operations when there may or may not be native support.
 #ifdef ARROW_USE_NATIVE_INT128
-  const __uint128_t x = (static_cast<__uint128_t>(x_hi) << 64) | x_lo;
-  const __uint128_t y = (static_cast<__uint128_t>(y_hi) << 64) | y_lo;
-  const __uint128_t r = x * y;
-  *lo = r & kInt64Mask;
-  *hi = r >> 64;
+struct uint128_t {
+  uint128_t() {}
+  uint128_t(uint64_t hi, uint64_t lo) : val_((static_cast<__uint128_t>(hi) << 64) | lo) {}
+  uint128_t(const BasicDecimal128& decimal) {
+    val_ = (static_cast<__uint128_t>(decimal.high_bits()) << 64) | decimal.low_bits();
+  }
+
+  uint64_t hi() { return val_ >> 64; }
+  uint64_t lo() { return val_ & kInt64Mask; }
+
+  uint128_t& operator+=(const uint128_t& other) {
+    val_ += other.val_;
+    return *this;
+  }
+
+  __uint128_t val_;
+};
+
+uint128_t operator*(const uint128_t& left, const uint128_t& right) {

Review comment:
       This (or perhaps some other change I made) seems to have improved performance significantly, it takes 13 ns~ with native int128 and 40 ns~ with uint64 fallback.

##########
File path: cpp/src/arrow/util/basic_decimal.cc
##########
@@ -254,67 +252,126 @@ BasicDecimal128& BasicDecimal128::operator>>=(uint32_t bits) {
 
 namespace {
 
-// TODO: Remove this guard once it's used by BasicDecimal256
-#ifndef ARROW_USE_NATIVE_INT128
-// This method losslessly multiplies x and y into a 128 bit unsigned integer
-// whose high bits will be stored in hi and low bits in lo.
-void ExtendAndMultiplyUint64(uint64_t x, uint64_t y, uint64_t* hi, uint64_t* lo) {
-#ifdef ARROW_USE_NATIVE_INT128
-  const __uint128_t r = static_cast<__uint128_t>(x) * y;
-  *lo = r & kInt64Mask;
-  *hi = r >> 64;
-#else
-  // If we can't use a native fallback, perform multiplication
-  // by splitting up x and y into 32 bit high/low bit components,
+// Multiply two N bit word components into a 2*N bit result, with high bits
+// stored in hi and low bits in lo.
+template <typename Word>
+inline void ExtendAndMultiplyUint(Word x, Word y, Word* hi, Word* lo) {
+  // Perform multiplication on two N bit words x and y into a 2*N bit result
+  // by splitting up x and y into N/2 bit high/low bit components,
   // allowing us to represent the multiplication as
-  // x * y = x_lo * y_lo + x_hi * y_lo * 2^32 + y_hi * x_lo * 2^32
-  // + x_hi * y_hi * 2^64.
+  // x * y = x_lo * y_lo + x_hi * y_lo * 2^N/2 + y_hi * x_lo * 2^N/2
+  // + x_hi * y_hi * 2^N
   //
-  // Now, consider the final output as lo_lo || lo_hi || hi_lo || hi_hi.
+  // Now, consider the final output as lo_lo || lo_hi || hi_lo || hi_hi
   // Therefore,
   // lo_lo is (x_lo * y_lo)_lo,
   // lo_hi is ((x_lo * y_lo)_hi + (x_hi * y_lo)_lo + (x_lo * y_hi)_lo)_lo,
   // hi_lo is ((x_hi * y_hi)_lo + (x_hi * y_lo)_hi + (x_lo * y_hi)_hi)_hi,
   // hi_hi is (x_hi * y_hi)_hi
-  const uint64_t x_lo = x & kIntMask;
-  const uint64_t y_lo = y & kIntMask;
-  const uint64_t x_hi = x >> 32;
-  const uint64_t y_hi = y >> 32;
+  constexpr Word kHighBitShift = sizeof(Word) * 4;
+  constexpr Word kLowBitMask = (static_cast<Word>(1) << kHighBitShift) - 1;
 
-  const uint64_t t = x_lo * y_lo;
-  const uint64_t t_lo = t & kIntMask;
-  const uint64_t t_hi = t >> 32;
+  const Word x_lo = x & kLowBitMask;
+  const Word y_lo = y & kLowBitMask;
+  const Word x_hi = x >> kHighBitShift;
+  const Word y_hi = y >> kHighBitShift;
 
-  const uint64_t u = x_hi * y_lo + t_hi;
-  const uint64_t u_lo = u & kIntMask;
-  const uint64_t u_hi = u >> 32;
+  const Word t = x_lo * y_lo;
+  const Word t_lo = t & kLowBitMask;
+  const Word t_hi = t >> kHighBitShift;
 
-  const uint64_t v = x_lo * y_hi + u_lo;
-  const uint64_t v_hi = v >> 32;
+  const Word u = x_hi * y_lo + t_hi;
+  const Word u_lo = u & kLowBitMask;
+  const Word u_hi = u >> kHighBitShift;
+
+  const Word v = x_lo * y_hi + u_lo;
+  const Word v_hi = v >> kHighBitShift;
 
   *hi = x_hi * y_hi + u_hi + v_hi;
-  *lo = (v << 32) | t_lo;
-#endif
+  *lo = (v << kHighBitShift) + t_lo;
 }
-#endif
 
-void MultiplyUint128(uint64_t x_hi, uint64_t x_lo, uint64_t y_hi, uint64_t y_lo,
-                     uint64_t* hi, uint64_t* lo) {
+// Convenience wrapper type over 128 bit unsigned integers. We opt not to
+// replace the uint128_t type in int128_internal.h because it would require
+// significantly more implementation work to be done. This class merely
+// provides the minimum necessary set of functions to perform 128+ bit
+// multiplication operations when there may or may not be native support.
 #ifdef ARROW_USE_NATIVE_INT128
-  const __uint128_t x = (static_cast<__uint128_t>(x_hi) << 64) | x_lo;
-  const __uint128_t y = (static_cast<__uint128_t>(y_hi) << 64) | y_lo;
-  const __uint128_t r = x * y;
-  *lo = r & kInt64Mask;
-  *hi = r >> 64;
+struct uint128_t {
+  uint128_t() {}
+  uint128_t(uint64_t hi, uint64_t lo) : val_((static_cast<__uint128_t>(hi) << 64) | lo) {}
+  uint128_t(const BasicDecimal128& decimal) {
+    val_ = (static_cast<__uint128_t>(decimal.high_bits()) << 64) | decimal.low_bits();
+  }
+
+  uint64_t hi() { return val_ >> 64; }
+  uint64_t lo() { return val_ & kInt64Mask; }
+
+  uint128_t& operator+=(const uint128_t& other) {
+    val_ += other.val_;
+    return *this;
+  }
+
+  __uint128_t val_;
+};
+
+uint128_t operator*(const uint128_t& left, const uint128_t& right) {
+  uint128_t r;
+  r.val_ = left.val_ * right.val_;
+  return r;
+}
 #else
-  // To perform 128 bit multiplication without a native fallback
-  // we first perform lossless 64 bit multiplication of the low
-  // bits, and then add x_hi * y_lo and x_lo * y_hi to the high
-  // bits. Note that we can skip adding x_hi * y_hi because it
-  // always will be over 128 bits.
-  ExtendAndMultiplyUint64(x_lo, y_lo, hi, lo);
-  *hi += (x_hi * y_lo) + (x_lo * y_hi);
+struct uint128_t {
+  uint128_t() {}
+  uint128_t(uint64_t hi, uint64_t lo) : hi_(hi), lo_(lo) {}
+  uint128_t(const BasicDecimal128& decimal) {
+    hi_ = decimal.high_bits();
+    lo_ = decimal.low_bits();
+  }
+
+  uint64_t hi() const { return hi_; }
+  uint64_t lo() const { return lo_; }
+
+  uint128_t& operator+=(const uint128_t& other) {
+    // To deduce the carry bit, we perform "65 bit" addition on the low bits and
+    // seeing if the resulting high bit is 1. This is accomplished by shifting the
+    // low bits to the right by 1 (chopping off the lowest bit), then adding 1 if the
+    // result of adding the two chopped bits would have produced a carry.
+    uint64_t carry = (((lo_ & other.lo_) & 1) + (lo_ >> 1) + (other.lo_ >> 1)) >> 63;
+    hi_ += other.hi_ + carry;
+    lo_ += other.lo_;
+    return *this;
+  }
+
+  uint64_t hi_;
+  uint64_t lo_;
+};
+
+uint128_t operator*(const uint128_t& left, const uint128_t& right) {
+  uint128_t r;
+  ExtendAndMultiplyUint(left.lo_, right.lo_, &r.hi_, &r.lo_);
+  r.hi_ += (left.hi_ * right.lo_) + (left.lo_ * right.hi_);
+  return r;
+}
 #endif
+
+// Multiplies two N * 64 bit unsigned integer types, represented by a uint64_t
+// array into a same sized output. Overflow in multiplication is considered UB

Review comment:
       Undefined Behavior, clarified in comments.




----------------------------------------------------------------
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] Luminarys commented on pull request #8344: Add BasicDecimal256 Multiplication Support (PR for decimal256 branch, not master)

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


   I'll take a closer look tommorow, but we should also wait for feedback from @MingyuZhong before proceeding.


----------------------------------------------------------------
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] Luminarys commented on pull request #8344: Add BasicDecimal256 Multiplication Support (PR for decimal256 branch, not master)

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


   It turns out one of the checks is due to a compiler bug in Clang, I've tweaked the definition structure of the BasicDecimal256 header to handle 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] MingyuZhong commented on a change in pull request #8344: Add BasicDecimal256 Multiplication Support (PR for decimal256 branch, not master)

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



##########
File path: cpp/src/arrow/util/basic_decimal.cc
##########
@@ -254,67 +254,125 @@ BasicDecimal128& BasicDecimal128::operator>>=(uint32_t bits) {
 
 namespace {
 
-// TODO: Remove this guard once it's used by BasicDecimal256
-#ifndef ARROW_USE_NATIVE_INT128
-// This method losslessly multiplies x and y into a 128 bit unsigned integer
-// whose high bits will be stored in hi and low bits in lo.
-void ExtendAndMultiplyUint64(uint64_t x, uint64_t y, uint64_t* hi, uint64_t* lo) {
+// Convenience wrapper type over 128 bit unsigned integers. We opt not to
+// replace the uint128_t type in int128_internal.h because it would require
+// significantly more implementation work to be done. This class merely
+// provides the minimum necessary set of functions to perform 128+ bit
+// multiplication operations when there may or may not be native support.
 #ifdef ARROW_USE_NATIVE_INT128
-  const __uint128_t r = static_cast<__uint128_t>(x) * y;
-  *lo = r & kInt64Mask;
-  *hi = r >> 64;
+struct uint128_t {
+  uint128_t() {}
+  uint128_t(uint64_t hi, uint64_t lo) : val_((static_cast<__uint128_t>(hi) << 64) | lo) {}
+  uint128_t(const BasicDecimal128& decimal) {
+    val_ = (static_cast<__uint128_t>(decimal.high_bits()) << 64) | decimal.low_bits();
+  }
+
+  uint64_t hi() { return val_ >> 64; }
+  uint64_t lo() { return val_ & kInt64Mask; }
+
+  uint128_t& operator+=(const uint128_t& other) {
+    val_ += other.val_;
+    return *this;
+  }
+
+  uint128_t& operator*=(const uint128_t& other) {
+    val_ *= other.val_;
+    return *this;
+  }
+
+  __uint128_t val_;
+};
+
 #else
-  // If we can't use a native fallback, perform multiplication
+// Multiply two 64 bit word components into a 128 bit result, with high bits
+// stored in hi and low bits in lo.
+inline void ExtendAndMultiply(uint64_t x, uint64_t y, uint64_t* hi, uint64_t* lo) {
+  // Perform multiplication on two 64 bit words x and y into a 128 bit result
   // by splitting up x and y into 32 bit high/low bit components,
   // allowing us to represent the multiplication as
   // x * y = x_lo * y_lo + x_hi * y_lo * 2^32 + y_hi * x_lo * 2^32
-  // + x_hi * y_hi * 2^64.
+  // + x_hi * y_hi * 2^64
   //
-  // Now, consider the final output as lo_lo || lo_hi || hi_lo || hi_hi.
+  // Now, consider the final output as lo_lo || lo_hi || hi_lo || hi_hi
   // Therefore,
   // lo_lo is (x_lo * y_lo)_lo,
   // lo_hi is ((x_lo * y_lo)_hi + (x_hi * y_lo)_lo + (x_lo * y_hi)_lo)_lo,
   // hi_lo is ((x_hi * y_hi)_lo + (x_hi * y_lo)_hi + (x_lo * y_hi)_hi)_hi,
   // hi_hi is (x_hi * y_hi)_hi
-  const uint64_t x_lo = x & kIntMask;
-  const uint64_t y_lo = y & kIntMask;
+  const uint64_t x_lo = x & kInt32Mask;
+  const uint64_t y_lo = y & kInt32Mask;
   const uint64_t x_hi = x >> 32;
   const uint64_t y_hi = y >> 32;
 
   const uint64_t t = x_lo * y_lo;
-  const uint64_t t_lo = t & kIntMask;
+  const uint64_t t_lo = t & kInt32Mask;
   const uint64_t t_hi = t >> 32;
 
   const uint64_t u = x_hi * y_lo + t_hi;
-  const uint64_t u_lo = u & kIntMask;
+  const uint64_t u_lo = u & kInt32Mask;
   const uint64_t u_hi = u >> 32;
 
   const uint64_t v = x_lo * y_hi + u_lo;
   const uint64_t v_hi = v >> 32;
 
   *hi = x_hi * y_hi + u_hi + v_hi;
-  *lo = (v << 32) | t_lo;
-#endif
+  *lo = (v << 32) + t_lo;
 }
-#endif
 
-void MultiplyUint128(uint64_t x_hi, uint64_t x_lo, uint64_t y_hi, uint64_t y_lo,
-                     uint64_t* hi, uint64_t* lo) {
-#ifdef ARROW_USE_NATIVE_INT128
-  const __uint128_t x = (static_cast<__uint128_t>(x_hi) << 64) | x_lo;
-  const __uint128_t y = (static_cast<__uint128_t>(y_hi) << 64) | y_lo;
-  const __uint128_t r = x * y;
-  *lo = r & kInt64Mask;
-  *hi = r >> 64;
-#else
-  // To perform 128 bit multiplication without a native fallback
-  // we first perform lossless 64 bit multiplication of the low
-  // bits, and then add x_hi * y_lo and x_lo * y_hi to the high
-  // bits. Note that we can skip adding x_hi * y_hi because it
-  // always will be over 128 bits.
-  ExtendAndMultiplyUint64(x_lo, y_lo, hi, lo);
-  *hi += (x_hi * y_lo) + (x_lo * y_hi);
+struct uint128_t {
+  uint128_t() {}
+  uint128_t(uint64_t hi, uint64_t lo) : hi_(hi), lo_(lo) {}
+  uint128_t(const BasicDecimal128& decimal) {
+    hi_ = decimal.high_bits();
+    lo_ = decimal.low_bits();
+  }
+
+  uint64_t hi() const { return hi_; }
+  uint64_t lo() const { return lo_; }
+
+  uint128_t& operator+=(const uint128_t& other) {
+    // To deduce the carry bit, we perform "65 bit" addition on the low bits and
+    // seeing if the resulting high bit is 1. This is accomplished by shifting the
+    // low bits to the right by 1 (chopping off the lowest bit), then adding 1 if the
+    // result of adding the two chopped bits would have produced a carry.
+    uint64_t carry = (((lo_ & other.lo_) & 1) + (lo_ >> 1) + (other.lo_ >> 1)) >> 63;
+    hi_ += other.hi_ + carry;
+    lo_ += other.lo_;
+    return *this;
+  }
+
+  uint128_t& operator*=(const uint128_t& other) {
+    uint128_t r;
+    ExtendAndMultiply(lo_, other.lo_, &r.hi_, &r.lo_);
+    r.hi_ += (hi_ * other.lo_) + (lo_ * other.hi_);
+    *this = r;
+    return *this;
+  }
+
+  uint64_t hi_;
+  uint64_t lo_;
+};
 #endif
+
+// Multiplies two N * 64 bit unsigned integer types, represented by a uint64_t
+// array into a same sized output. Elements in the array should be in
+// little endian order, and output will be the same. Overflow in multiplication
+// is considered undefined behavior and will not be reported.

Review comment:
       Is it really undefined? Isn't the output the lower N * 64 bits of the actual result?




----------------------------------------------------------------
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 #8344: Add BasicDecimal256 Multiplication Support

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


   <!--
     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.
   -->
   
   Thanks for opening a pull request!
   
   Could you open an issue for this pull request on JIRA?
   https://issues.apache.org/jira/browse/ARROW
   
   Then could you also rename pull request title in the following format?
   
       ARROW-${JIRA_ID}: [${COMPONENT}] ${SUMMARY}
   
   See also:
   
     * [Other pull requests](https://github.com/apache/arrow/pulls/)
     * [Contribution Guidelines - How to contribute patches](https://arrow.apache.org/docs/developers/contributing.html#how-to-contribute-patches)
   


----------------------------------------------------------------
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] Luminarys commented on a change in pull request #8344: Add BasicDecimal256 Multiplication Support (PR for decimal256 branch, not master)

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



##########
File path: cpp/src/arrow/util/basic_decimal.cc
##########
@@ -254,67 +252,121 @@ BasicDecimal128& BasicDecimal128::operator>>=(uint32_t bits) {
 
 namespace {
 
-// TODO: Remove this guard once it's used by BasicDecimal256
-#ifndef ARROW_USE_NATIVE_INT128
-// This method losslessly multiplies x and y into a 128 bit unsigned integer
-// whose high bits will be stored in hi and low bits in lo.
-void ExtendAndMultiplyUint64(uint64_t x, uint64_t y, uint64_t* hi, uint64_t* lo) {
-#ifdef ARROW_USE_NATIVE_INT128
-  const __uint128_t r = static_cast<__uint128_t>(x) * y;
-  *lo = r & kInt64Mask;
-  *hi = r >> 64;
-#else
-  // If we can't use a native fallback, perform multiplication
-  // by splitting up x and y into 32 bit high/low bit components,
+// Multiply two N bit word components into a 2*N bit result, with high bits
+// stored in hi and low bits in lo.
+template <typename Word>
+void ExtendAndMultiplyUint(Word x, Word y, Word* hi, Word* lo) {
+  // Perform multiplication on two N bit words x and y into a 2*N bit result
+  // by splitting up x and y into N/2 bit high/low bit components,
   // allowing us to represent the multiplication as
-  // x * y = x_lo * y_lo + x_hi * y_lo * 2^32 + y_hi * x_lo * 2^32
-  // + x_hi * y_hi * 2^64.
+  // x * y = x_lo * y_lo + x_hi * y_lo * 2^N/2 + y_hi * x_lo * 2^N/2
+  // + x_hi * y_hi * 2^N
   //
-  // Now, consider the final output as lo_lo || lo_hi || hi_lo || hi_hi.
+  // Now, consider the final output as lo_lo || lo_hi || hi_lo || hi_hi
   // Therefore,
   // lo_lo is (x_lo * y_lo)_lo,
   // lo_hi is ((x_lo * y_lo)_hi + (x_hi * y_lo)_lo + (x_lo * y_hi)_lo)_lo,
   // hi_lo is ((x_hi * y_hi)_lo + (x_hi * y_lo)_hi + (x_lo * y_hi)_hi)_hi,
   // hi_hi is (x_hi * y_hi)_hi
-  const uint64_t x_lo = x & kIntMask;
-  const uint64_t y_lo = y & kIntMask;
-  const uint64_t x_hi = x >> 32;
-  const uint64_t y_hi = y >> 32;
+  constexpr Word kHighBitShift = sizeof(Word) * 4;
+  constexpr Word kLowBitMask = (static_cast<Word>(1) << kHighBitShift) - 1;
 
-  const uint64_t t = x_lo * y_lo;
-  const uint64_t t_lo = t & kIntMask;
-  const uint64_t t_hi = t >> 32;
+  const Word x_lo = x & kLowBitMask;
+  const Word y_lo = y & kLowBitMask;
+  const Word x_hi = x >> kHighBitShift;
+  const Word y_hi = y >> kHighBitShift;
 
-  const uint64_t u = x_hi * y_lo + t_hi;
-  const uint64_t u_lo = u & kIntMask;
-  const uint64_t u_hi = u >> 32;
+  const Word t = x_lo * y_lo;
+  const Word t_lo = t & kLowBitMask;
+  const Word t_hi = t >> kHighBitShift;
 
-  const uint64_t v = x_lo * y_hi + u_lo;
-  const uint64_t v_hi = v >> 32;
+  const Word u = x_hi * y_lo + t_hi;
+  const Word u_lo = u & kLowBitMask;
+  const Word u_hi = u >> kHighBitShift;
+
+  const Word v = x_lo * y_hi + u_lo;
+  const Word v_hi = v >> kHighBitShift;
 
   *hi = x_hi * y_hi + u_hi + v_hi;
-  *lo = (v << 32) | t_lo;
-#endif
+  *lo = (v << kHighBitShift) + t_lo;
 }
-#endif
 
-void MultiplyUint128(uint64_t x_hi, uint64_t x_lo, uint64_t y_hi, uint64_t y_lo,
-                     uint64_t* hi, uint64_t* lo) {
+// Convenience wrapper type over 128 bit unsigned integers
 #ifdef ARROW_USE_NATIVE_INT128
-  const __uint128_t x = (static_cast<__uint128_t>(x_hi) << 64) | x_lo;
-  const __uint128_t y = (static_cast<__uint128_t>(y_hi) << 64) | y_lo;
-  const __uint128_t r = x * y;
-  *lo = r & kInt64Mask;
-  *hi = r >> 64;
+struct uint128_t {
+  uint128_t() {}
+  uint128_t(uint64_t hi, uint64_t lo) : val_((static_cast<__uint128_t>(hi) << 64) | lo) {}
+  uint128_t(const BasicDecimal128& decimal) {
+    val_ = (static_cast<__uint128_t>(decimal.high_bits()) << 64) | decimal.low_bits();
+  }
+
+  uint64_t hi() { return val_ >> 64; }
+  uint64_t lo() { return val_ & kInt64Mask; }
+
+  uint128_t& operator+=(const uint128_t& other) {
+    val_ += other.val_;
+    return *this;
+  }
+
+  __uint128_t val_;
+};
+
+uint128_t operator*(const uint128_t& left, const uint128_t& right) {
+  uint128_t r;
+  r.val_ = left.val_ * right.val_;
+  return r;
+}
 #else
-  // To perform 128 bit multiplication without a native fallback
-  // we first perform lossless 64 bit multiplication of the low
-  // bits, and then add x_hi * y_lo and x_lo * y_hi to the high
-  // bits. Note that we can skip adding x_hi * y_hi because it
-  // always will be over 128 bits.
-  ExtendAndMultiplyUint64(x_lo, y_lo, hi, lo);
-  *hi += (x_hi * y_lo) + (x_lo * y_hi);
+struct uint128_t {
+  uint128_t() {}
+  uint128_t(uint64_t hi, uint64_t lo) : hi_(hi), lo_(lo) {}
+  uint128_t(const BasicDecimal128& decimal) {
+    hi_ = decimal.high_bits();
+    lo_ = decimal.low_bits();
+  }
+
+  uint64_t hi() const { return hi_; }
+  uint64_t lo() const { return lo_; }
+
+  uint128_t& operator+=(const uint128_t& other) {
+    // To deduce the carry bit, we perform "65 bit" addition on the low bits and
+    // seeing if the resulting high bit is 1. This is accomplished by shifting the
+    // low bits to the right by 1 (chopping off the lowest bit), then adding 1 if the
+    // result of adding the two chopped bits would have produced a carry.
+    uint64_t carry = (((lo_ & other.lo_) & 1) + (lo_ >> 1) + (other.lo_ >> 1)) >> 63;
+    hi_ += other.hi_ + carry;
+    lo_ += other.lo_;
+    return *this;
+  }
+
+  uint64_t hi_;
+  uint64_t lo_;
+};
+
+uint128_t operator*(const uint128_t& left, const uint128_t& right) {
+  uint128_t r;
+  ExtendAndMultiplyUint(left.lo_, right.lo_, &r.hi_, &r.lo_);
+  r.hi_ += (left.hi_ * right.lo_) + (left.lo_ * right.hi_);
+  return r;
+}
 #endif
+
+template <int N>
+inline void PartialMultiplyOverflow(const std::array<uint64_t, N>& lh,

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] MingyuZhong commented on a change in pull request #8344: Add BasicDecimal256 Multiplication Support (PR for decimal256 branch, not master)

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



##########
File path: cpp/src/arrow/util/basic_decimal.cc
##########
@@ -254,67 +252,121 @@ BasicDecimal128& BasicDecimal128::operator>>=(uint32_t bits) {
 
 namespace {
 
-// TODO: Remove this guard once it's used by BasicDecimal256
-#ifndef ARROW_USE_NATIVE_INT128
-// This method losslessly multiplies x and y into a 128 bit unsigned integer
-// whose high bits will be stored in hi and low bits in lo.
-void ExtendAndMultiplyUint64(uint64_t x, uint64_t y, uint64_t* hi, uint64_t* lo) {
-#ifdef ARROW_USE_NATIVE_INT128
-  const __uint128_t r = static_cast<__uint128_t>(x) * y;
-  *lo = r & kInt64Mask;
-  *hi = r >> 64;
-#else
-  // If we can't use a native fallback, perform multiplication
-  // by splitting up x and y into 32 bit high/low bit components,
+// Multiply two N bit word components into a 2*N bit result, with high bits
+// stored in hi and low bits in lo.
+template <typename Word>
+void ExtendAndMultiplyUint(Word x, Word y, Word* hi, Word* lo) {
+  // Perform multiplication on two N bit words x and y into a 2*N bit result
+  // by splitting up x and y into N/2 bit high/low bit components,
   // allowing us to represent the multiplication as
-  // x * y = x_lo * y_lo + x_hi * y_lo * 2^32 + y_hi * x_lo * 2^32
-  // + x_hi * y_hi * 2^64.
+  // x * y = x_lo * y_lo + x_hi * y_lo * 2^N/2 + y_hi * x_lo * 2^N/2
+  // + x_hi * y_hi * 2^N
   //
-  // Now, consider the final output as lo_lo || lo_hi || hi_lo || hi_hi.
+  // Now, consider the final output as lo_lo || lo_hi || hi_lo || hi_hi
   // Therefore,
   // lo_lo is (x_lo * y_lo)_lo,
   // lo_hi is ((x_lo * y_lo)_hi + (x_hi * y_lo)_lo + (x_lo * y_hi)_lo)_lo,
   // hi_lo is ((x_hi * y_hi)_lo + (x_hi * y_lo)_hi + (x_lo * y_hi)_hi)_hi,
   // hi_hi is (x_hi * y_hi)_hi
-  const uint64_t x_lo = x & kIntMask;
-  const uint64_t y_lo = y & kIntMask;
-  const uint64_t x_hi = x >> 32;
-  const uint64_t y_hi = y >> 32;
+  constexpr Word kHighBitShift = sizeof(Word) * 4;
+  constexpr Word kLowBitMask = (static_cast<Word>(1) << kHighBitShift) - 1;
 
-  const uint64_t t = x_lo * y_lo;
-  const uint64_t t_lo = t & kIntMask;
-  const uint64_t t_hi = t >> 32;
+  const Word x_lo = x & kLowBitMask;
+  const Word y_lo = y & kLowBitMask;
+  const Word x_hi = x >> kHighBitShift;
+  const Word y_hi = y >> kHighBitShift;
 
-  const uint64_t u = x_hi * y_lo + t_hi;
-  const uint64_t u_lo = u & kIntMask;
-  const uint64_t u_hi = u >> 32;
+  const Word t = x_lo * y_lo;
+  const Word t_lo = t & kLowBitMask;
+  const Word t_hi = t >> kHighBitShift;
 
-  const uint64_t v = x_lo * y_hi + u_lo;
-  const uint64_t v_hi = v >> 32;
+  const Word u = x_hi * y_lo + t_hi;
+  const Word u_lo = u & kLowBitMask;
+  const Word u_hi = u >> kHighBitShift;
+
+  const Word v = x_lo * y_hi + u_lo;
+  const Word v_hi = v >> kHighBitShift;
 
   *hi = x_hi * y_hi + u_hi + v_hi;
-  *lo = (v << 32) | t_lo;
-#endif
+  *lo = (v << kHighBitShift) + t_lo;
 }
-#endif
 
-void MultiplyUint128(uint64_t x_hi, uint64_t x_lo, uint64_t y_hi, uint64_t y_lo,
-                     uint64_t* hi, uint64_t* lo) {
+// Convenience wrapper type over 128 bit unsigned integers
 #ifdef ARROW_USE_NATIVE_INT128
-  const __uint128_t x = (static_cast<__uint128_t>(x_hi) << 64) | x_lo;
-  const __uint128_t y = (static_cast<__uint128_t>(y_hi) << 64) | y_lo;
-  const __uint128_t r = x * y;
-  *lo = r & kInt64Mask;
-  *hi = r >> 64;
+struct uint128_t {
+  uint128_t() {}
+  uint128_t(uint64_t hi, uint64_t lo) : val_((static_cast<__uint128_t>(hi) << 64) | lo) {}
+  uint128_t(const BasicDecimal128& decimal) {
+    val_ = (static_cast<__uint128_t>(decimal.high_bits()) << 64) | decimal.low_bits();
+  }
+
+  uint64_t hi() { return val_ >> 64; }
+  uint64_t lo() { return val_ & kInt64Mask; }
+
+  uint128_t& operator+=(const uint128_t& other) {
+    val_ += other.val_;
+    return *this;
+  }
+
+  __uint128_t val_;
+};
+
+uint128_t operator*(const uint128_t& left, const uint128_t& right) {
+  uint128_t r;
+  r.val_ = left.val_ * right.val_;
+  return r;
+}
 #else
-  // To perform 128 bit multiplication without a native fallback
-  // we first perform lossless 64 bit multiplication of the low
-  // bits, and then add x_hi * y_lo and x_lo * y_hi to the high
-  // bits. Note that we can skip adding x_hi * y_hi because it
-  // always will be over 128 bits.
-  ExtendAndMultiplyUint64(x_lo, y_lo, hi, lo);
-  *hi += (x_hi * y_lo) + (x_lo * y_hi);
+struct uint128_t {
+  uint128_t() {}
+  uint128_t(uint64_t hi, uint64_t lo) : hi_(hi), lo_(lo) {}
+  uint128_t(const BasicDecimal128& decimal) {
+    hi_ = decimal.high_bits();
+    lo_ = decimal.low_bits();
+  }
+
+  uint64_t hi() const { return hi_; }
+  uint64_t lo() const { return lo_; }
+
+  uint128_t& operator+=(const uint128_t& other) {
+    // To deduce the carry bit, we perform "65 bit" addition on the low bits and
+    // seeing if the resulting high bit is 1. This is accomplished by shifting the
+    // low bits to the right by 1 (chopping off the lowest bit), then adding 1 if the
+    // result of adding the two chopped bits would have produced a carry.
+    uint64_t carry = (((lo_ & other.lo_) & 1) + (lo_ >> 1) + (other.lo_ >> 1)) >> 63;
+    hi_ += other.hi_ + carry;
+    lo_ += other.lo_;
+    return *this;
+  }
+
+  uint64_t hi_;
+  uint64_t lo_;
+};
+
+uint128_t operator*(const uint128_t& left, const uint128_t& right) {
+  uint128_t r;
+  ExtendAndMultiplyUint(left.lo_, right.lo_, &r.hi_, &r.lo_);
+  r.hi_ += (left.hi_ * right.lo_) + (left.lo_ * right.hi_);
+  return r;
+}
 #endif
+
+template <int N>
+inline void PartialMultiplyOverflow(const std::array<uint64_t, N>& lh,

Review comment:
       This function does not tell whether this is overflow, so please rename it Multiply().

##########
File path: cpp/src/arrow/util/basic_decimal.cc
##########
@@ -254,69 +252,148 @@ BasicDecimal128& BasicDecimal128::operator>>=(uint32_t bits) {
 
 namespace {
 
-// TODO: Remove this guard once it's used by BasicDecimal256
-#ifndef ARROW_USE_NATIVE_INT128
-// This method losslessly multiplies x and y into a 128 bit unsigned integer
-// whose high bits will be stored in hi and low bits in lo.
-void ExtendAndMultiplyUint64(uint64_t x, uint64_t y, uint64_t* hi, uint64_t* lo) {
-#ifdef ARROW_USE_NATIVE_INT128
-  const __uint128_t r = static_cast<__uint128_t>(x) * y;
-  *lo = r & kInt64Mask;
-  *hi = r >> 64;
-#else
-  // If we can't use a native fallback, perform multiplication
-  // by splitting up x and y into 32 bit high/low bit components,
+// Multiply two N bit word components into a 2*N bit result, with high bits
+// stored in hi and low bits in lo.
+template <typename Word>
+void ExtendAndMultiplyUint(Word x, Word y, Word* hi, Word* lo) {

Review comment:
       Can you try making ExtendAndMultiplyUint inline?

##########
File path: cpp/src/arrow/util/basic_decimal.cc
##########
@@ -254,69 +252,148 @@ BasicDecimal128& BasicDecimal128::operator>>=(uint32_t bits) {
 
 namespace {
 
-// TODO: Remove this guard once it's used by BasicDecimal256
-#ifndef ARROW_USE_NATIVE_INT128
-// This method losslessly multiplies x and y into a 128 bit unsigned integer
-// whose high bits will be stored in hi and low bits in lo.
-void ExtendAndMultiplyUint64(uint64_t x, uint64_t y, uint64_t* hi, uint64_t* lo) {
-#ifdef ARROW_USE_NATIVE_INT128
-  const __uint128_t r = static_cast<__uint128_t>(x) * y;
-  *lo = r & kInt64Mask;
-  *hi = r >> 64;
-#else
-  // If we can't use a native fallback, perform multiplication
-  // by splitting up x and y into 32 bit high/low bit components,
+// Multiply two N bit word components into a 2*N bit result, with high bits
+// stored in hi and low bits in lo.
+template <typename Word>
+void ExtendAndMultiplyUint(Word x, Word y, Word* hi, Word* lo) {
+  // Perform multiplication on two N bit words x and y into a 2*N bit result
+  // by splitting up x and y into N/2 bit high/low bit components,
   // allowing us to represent the multiplication as
-  // x * y = x_lo * y_lo + x_hi * y_lo * 2^32 + y_hi * x_lo * 2^32
-  // + x_hi * y_hi * 2^64.
+  // x * y = x_lo * y_lo + x_hi * y_lo * 2^N/2 + y_hi * x_lo * 2^N/2
+  // + x_hi * y_hi * 2^N
   //
-  // Now, consider the final output as lo_lo || lo_hi || hi_lo || hi_hi.
+  // Now, consider the final output as lo_lo || lo_hi || hi_lo || hi_hi
   // Therefore,
   // lo_lo is (x_lo * y_lo)_lo,
   // lo_hi is ((x_lo * y_lo)_hi + (x_hi * y_lo)_lo + (x_lo * y_hi)_lo)_lo,
   // hi_lo is ((x_hi * y_hi)_lo + (x_hi * y_lo)_hi + (x_lo * y_hi)_hi)_hi,
   // hi_hi is (x_hi * y_hi)_hi
-  const uint64_t x_lo = x & kIntMask;
-  const uint64_t y_lo = y & kIntMask;
-  const uint64_t x_hi = x >> 32;
-  const uint64_t y_hi = y >> 32;
+  constexpr Word kHighBitShift = sizeof(Word) * 4;
+  constexpr Word kLowBitMask = (static_cast<Word>(1) << kHighBitShift) - 1;
 
-  const uint64_t t = x_lo * y_lo;
-  const uint64_t t_lo = t & kIntMask;
-  const uint64_t t_hi = t >> 32;
+  const Word x_lo = x & kLowBitMask;
+  const Word y_lo = y & kLowBitMask;
+  const Word x_hi = x >> kHighBitShift;
+  const Word y_hi = y >> kHighBitShift;
 
-  const uint64_t u = x_hi * y_lo + t_hi;
-  const uint64_t u_lo = u & kIntMask;
-  const uint64_t u_hi = u >> 32;
+  const Word t = x_lo * y_lo;
+  const Word t_lo = t & kLowBitMask;
+  const Word t_hi = t >> kHighBitShift;
 
-  const uint64_t v = x_lo * y_hi + u_lo;
-  const uint64_t v_hi = v >> 32;
+  const Word u = x_hi * y_lo + t_hi;
+  const Word u_lo = u & kLowBitMask;
+  const Word u_hi = u >> kHighBitShift;
+
+  const Word v = x_lo * y_hi + u_lo;
+  const Word v_hi = v >> kHighBitShift;
 
   *hi = x_hi * y_hi + u_hi + v_hi;
-  *lo = (v << 32) | t_lo;
-#endif
+  *lo = (v << kHighBitShift) + t_lo;
+}
+
+// Convenience wrapper type over 128 bit unsigned integers
+#ifdef ARROW_USE_NATIVE_INT128
+struct uint128_t {

Review comment:
       OK, please add a comment.




----------------------------------------------------------------
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] MingyuZhong commented on a change in pull request #8344: Add BasicDecimal256 Multiplication Support (PR for decimal256 branch, not master)

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



##########
File path: cpp/src/arrow/util/basic_decimal.cc
##########
@@ -254,67 +252,126 @@ BasicDecimal128& BasicDecimal128::operator>>=(uint32_t bits) {
 
 namespace {
 
-// TODO: Remove this guard once it's used by BasicDecimal256
-#ifndef ARROW_USE_NATIVE_INT128
-// This method losslessly multiplies x and y into a 128 bit unsigned integer
-// whose high bits will be stored in hi and low bits in lo.
-void ExtendAndMultiplyUint64(uint64_t x, uint64_t y, uint64_t* hi, uint64_t* lo) {
-#ifdef ARROW_USE_NATIVE_INT128
-  const __uint128_t r = static_cast<__uint128_t>(x) * y;
-  *lo = r & kInt64Mask;
-  *hi = r >> 64;
-#else
-  // If we can't use a native fallback, perform multiplication
-  // by splitting up x and y into 32 bit high/low bit components,
+// Multiply two N bit word components into a 2*N bit result, with high bits
+// stored in hi and low bits in lo.
+template <typename Word>
+inline void ExtendAndMultiplyUint(Word x, Word y, Word* hi, Word* lo) {
+  // Perform multiplication on two N bit words x and y into a 2*N bit result
+  // by splitting up x and y into N/2 bit high/low bit components,
   // allowing us to represent the multiplication as
-  // x * y = x_lo * y_lo + x_hi * y_lo * 2^32 + y_hi * x_lo * 2^32
-  // + x_hi * y_hi * 2^64.
+  // x * y = x_lo * y_lo + x_hi * y_lo * 2^N/2 + y_hi * x_lo * 2^N/2
+  // + x_hi * y_hi * 2^N
   //
-  // Now, consider the final output as lo_lo || lo_hi || hi_lo || hi_hi.
+  // Now, consider the final output as lo_lo || lo_hi || hi_lo || hi_hi
   // Therefore,
   // lo_lo is (x_lo * y_lo)_lo,
   // lo_hi is ((x_lo * y_lo)_hi + (x_hi * y_lo)_lo + (x_lo * y_hi)_lo)_lo,
   // hi_lo is ((x_hi * y_hi)_lo + (x_hi * y_lo)_hi + (x_lo * y_hi)_hi)_hi,
   // hi_hi is (x_hi * y_hi)_hi
-  const uint64_t x_lo = x & kIntMask;
-  const uint64_t y_lo = y & kIntMask;
-  const uint64_t x_hi = x >> 32;
-  const uint64_t y_hi = y >> 32;
+  constexpr Word kHighBitShift = sizeof(Word) * 4;
+  constexpr Word kLowBitMask = (static_cast<Word>(1) << kHighBitShift) - 1;
 
-  const uint64_t t = x_lo * y_lo;
-  const uint64_t t_lo = t & kIntMask;
-  const uint64_t t_hi = t >> 32;
+  const Word x_lo = x & kLowBitMask;
+  const Word y_lo = y & kLowBitMask;
+  const Word x_hi = x >> kHighBitShift;
+  const Word y_hi = y >> kHighBitShift;
 
-  const uint64_t u = x_hi * y_lo + t_hi;
-  const uint64_t u_lo = u & kIntMask;
-  const uint64_t u_hi = u >> 32;
+  const Word t = x_lo * y_lo;
+  const Word t_lo = t & kLowBitMask;
+  const Word t_hi = t >> kHighBitShift;
 
-  const uint64_t v = x_lo * y_hi + u_lo;
-  const uint64_t v_hi = v >> 32;
+  const Word u = x_hi * y_lo + t_hi;
+  const Word u_lo = u & kLowBitMask;
+  const Word u_hi = u >> kHighBitShift;
+
+  const Word v = x_lo * y_hi + u_lo;
+  const Word v_hi = v >> kHighBitShift;
 
   *hi = x_hi * y_hi + u_hi + v_hi;
-  *lo = (v << 32) | t_lo;
-#endif
+  *lo = (v << kHighBitShift) + t_lo;
 }
-#endif
 
-void MultiplyUint128(uint64_t x_hi, uint64_t x_lo, uint64_t y_hi, uint64_t y_lo,
-                     uint64_t* hi, uint64_t* lo) {
+// Convenience wrapper type over 128 bit unsigned integers. We opt not to
+// replace the uint128_t type in int128_internal.h because it would require
+// significantly more implementation work to be done. This class merely
+// provides the minimum necessary set of functions to perform 128+ bit
+// multiplication operations when there may or may not be native support.
 #ifdef ARROW_USE_NATIVE_INT128
-  const __uint128_t x = (static_cast<__uint128_t>(x_hi) << 64) | x_lo;
-  const __uint128_t y = (static_cast<__uint128_t>(y_hi) << 64) | y_lo;
-  const __uint128_t r = x * y;
-  *lo = r & kInt64Mask;
-  *hi = r >> 64;
+struct uint128_t {
+  uint128_t() {}
+  uint128_t(uint64_t hi, uint64_t lo) : val_((static_cast<__uint128_t>(hi) << 64) | lo) {}
+  uint128_t(const BasicDecimal128& decimal) {
+    val_ = (static_cast<__uint128_t>(decimal.high_bits()) << 64) | decimal.low_bits();
+  }
+
+  uint64_t hi() { return val_ >> 64; }
+  uint64_t lo() { return val_ & kInt64Mask; }
+
+  uint128_t& operator+=(const uint128_t& other) {
+    val_ += other.val_;
+    return *this;
+  }
+
+  __uint128_t val_;
+};
+
+uint128_t operator*(const uint128_t& left, const uint128_t& right) {
+  uint128_t r;
+  r.val_ = left.val_ * right.val_;
+  return r;
+}
 #else
-  // To perform 128 bit multiplication without a native fallback
-  // we first perform lossless 64 bit multiplication of the low
-  // bits, and then add x_hi * y_lo and x_lo * y_hi to the high
-  // bits. Note that we can skip adding x_hi * y_hi because it
-  // always will be over 128 bits.
-  ExtendAndMultiplyUint64(x_lo, y_lo, hi, lo);
-  *hi += (x_hi * y_lo) + (x_lo * y_hi);
+struct uint128_t {
+  uint128_t() {}
+  uint128_t(uint64_t hi, uint64_t lo) : hi_(hi), lo_(lo) {}
+  uint128_t(const BasicDecimal128& decimal) {
+    hi_ = decimal.high_bits();
+    lo_ = decimal.low_bits();
+  }
+
+  uint64_t hi() const { return hi_; }
+  uint64_t lo() const { return lo_; }
+
+  uint128_t& operator+=(const uint128_t& other) {
+    // To deduce the carry bit, we perform "65 bit" addition on the low bits and
+    // seeing if the resulting high bit is 1. This is accomplished by shifting the
+    // low bits to the right by 1 (chopping off the lowest bit), then adding 1 if the
+    // result of adding the two chopped bits would have produced a carry.
+    uint64_t carry = (((lo_ & other.lo_) & 1) + (lo_ >> 1) + (other.lo_ >> 1)) >> 63;
+    hi_ += other.hi_ + carry;
+    lo_ += other.lo_;
+    return *this;
+  }
+
+  uint64_t hi_;
+  uint64_t lo_;
+};
+
+uint128_t operator*(const uint128_t& left, const uint128_t& right) {
+  uint128_t r;
+  ExtendAndMultiplyUint(left.lo_, right.lo_, &r.hi_, &r.lo_);
+  r.hi_ += (left.hi_ * right.lo_) + (left.lo_ * right.hi_);
+  return r;
+}
 #endif
+
+// Multiplies two N * 64 bit unsigned integer types, represented by a uint64_t

Review comment:
       Please comment that the elements in the array inputs and output have little-endian order.

##########
File path: cpp/src/arrow/util/basic_decimal.cc
##########
@@ -254,67 +252,126 @@ BasicDecimal128& BasicDecimal128::operator>>=(uint32_t bits) {
 
 namespace {
 
-// TODO: Remove this guard once it's used by BasicDecimal256
-#ifndef ARROW_USE_NATIVE_INT128
-// This method losslessly multiplies x and y into a 128 bit unsigned integer
-// whose high bits will be stored in hi and low bits in lo.
-void ExtendAndMultiplyUint64(uint64_t x, uint64_t y, uint64_t* hi, uint64_t* lo) {
-#ifdef ARROW_USE_NATIVE_INT128
-  const __uint128_t r = static_cast<__uint128_t>(x) * y;
-  *lo = r & kInt64Mask;
-  *hi = r >> 64;
-#else
-  // If we can't use a native fallback, perform multiplication
-  // by splitting up x and y into 32 bit high/low bit components,
+// Multiply two N bit word components into a 2*N bit result, with high bits
+// stored in hi and low bits in lo.
+template <typename Word>
+inline void ExtendAndMultiplyUint(Word x, Word y, Word* hi, Word* lo) {
+  // Perform multiplication on two N bit words x and y into a 2*N bit result
+  // by splitting up x and y into N/2 bit high/low bit components,
   // allowing us to represent the multiplication as
-  // x * y = x_lo * y_lo + x_hi * y_lo * 2^32 + y_hi * x_lo * 2^32
-  // + x_hi * y_hi * 2^64.
+  // x * y = x_lo * y_lo + x_hi * y_lo * 2^N/2 + y_hi * x_lo * 2^N/2
+  // + x_hi * y_hi * 2^N
   //
-  // Now, consider the final output as lo_lo || lo_hi || hi_lo || hi_hi.
+  // Now, consider the final output as lo_lo || lo_hi || hi_lo || hi_hi
   // Therefore,
   // lo_lo is (x_lo * y_lo)_lo,
   // lo_hi is ((x_lo * y_lo)_hi + (x_hi * y_lo)_lo + (x_lo * y_hi)_lo)_lo,
   // hi_lo is ((x_hi * y_hi)_lo + (x_hi * y_lo)_hi + (x_lo * y_hi)_hi)_hi,
   // hi_hi is (x_hi * y_hi)_hi
-  const uint64_t x_lo = x & kIntMask;
-  const uint64_t y_lo = y & kIntMask;
-  const uint64_t x_hi = x >> 32;
-  const uint64_t y_hi = y >> 32;
+  constexpr Word kHighBitShift = sizeof(Word) * 4;
+  constexpr Word kLowBitMask = (static_cast<Word>(1) << kHighBitShift) - 1;
 
-  const uint64_t t = x_lo * y_lo;
-  const uint64_t t_lo = t & kIntMask;
-  const uint64_t t_hi = t >> 32;
+  const Word x_lo = x & kLowBitMask;
+  const Word y_lo = y & kLowBitMask;
+  const Word x_hi = x >> kHighBitShift;
+  const Word y_hi = y >> kHighBitShift;
 
-  const uint64_t u = x_hi * y_lo + t_hi;
-  const uint64_t u_lo = u & kIntMask;
-  const uint64_t u_hi = u >> 32;
+  const Word t = x_lo * y_lo;
+  const Word t_lo = t & kLowBitMask;
+  const Word t_hi = t >> kHighBitShift;
 
-  const uint64_t v = x_lo * y_hi + u_lo;
-  const uint64_t v_hi = v >> 32;
+  const Word u = x_hi * y_lo + t_hi;
+  const Word u_lo = u & kLowBitMask;
+  const Word u_hi = u >> kHighBitShift;
+
+  const Word v = x_lo * y_hi + u_lo;
+  const Word v_hi = v >> kHighBitShift;
 
   *hi = x_hi * y_hi + u_hi + v_hi;
-  *lo = (v << 32) | t_lo;
-#endif
+  *lo = (v << kHighBitShift) + t_lo;
 }
-#endif
 
-void MultiplyUint128(uint64_t x_hi, uint64_t x_lo, uint64_t y_hi, uint64_t y_lo,
-                     uint64_t* hi, uint64_t* lo) {
+// Convenience wrapper type over 128 bit unsigned integers. We opt not to
+// replace the uint128_t type in int128_internal.h because it would require
+// significantly more implementation work to be done. This class merely
+// provides the minimum necessary set of functions to perform 128+ bit
+// multiplication operations when there may or may not be native support.
 #ifdef ARROW_USE_NATIVE_INT128
-  const __uint128_t x = (static_cast<__uint128_t>(x_hi) << 64) | x_lo;
-  const __uint128_t y = (static_cast<__uint128_t>(y_hi) << 64) | y_lo;
-  const __uint128_t r = x * y;
-  *lo = r & kInt64Mask;
-  *hi = r >> 64;
+struct uint128_t {
+  uint128_t() {}
+  uint128_t(uint64_t hi, uint64_t lo) : val_((static_cast<__uint128_t>(hi) << 64) | lo) {}
+  uint128_t(const BasicDecimal128& decimal) {
+    val_ = (static_cast<__uint128_t>(decimal.high_bits()) << 64) | decimal.low_bits();
+  }
+
+  uint64_t hi() { return val_ >> 64; }
+  uint64_t lo() { return val_ & kInt64Mask; }
+
+  uint128_t& operator+=(const uint128_t& other) {
+    val_ += other.val_;
+    return *this;
+  }
+
+  __uint128_t val_;
+};
+
+uint128_t operator*(const uint128_t& left, const uint128_t& right) {
+  uint128_t r;
+  r.val_ = left.val_ * right.val_;
+  return r;
+}
 #else
-  // To perform 128 bit multiplication without a native fallback
-  // we first perform lossless 64 bit multiplication of the low
-  // bits, and then add x_hi * y_lo and x_lo * y_hi to the high
-  // bits. Note that we can skip adding x_hi * y_hi because it
-  // always will be over 128 bits.
-  ExtendAndMultiplyUint64(x_lo, y_lo, hi, lo);
-  *hi += (x_hi * y_lo) + (x_lo * y_hi);
+struct uint128_t {
+  uint128_t() {}
+  uint128_t(uint64_t hi, uint64_t lo) : hi_(hi), lo_(lo) {}
+  uint128_t(const BasicDecimal128& decimal) {
+    hi_ = decimal.high_bits();
+    lo_ = decimal.low_bits();
+  }
+
+  uint64_t hi() const { return hi_; }
+  uint64_t lo() const { return lo_; }
+
+  uint128_t& operator+=(const uint128_t& other) {
+    // To deduce the carry bit, we perform "65 bit" addition on the low bits and
+    // seeing if the resulting high bit is 1. This is accomplished by shifting the
+    // low bits to the right by 1 (chopping off the lowest bit), then adding 1 if the
+    // result of adding the two chopped bits would have produced a carry.
+    uint64_t carry = (((lo_ & other.lo_) & 1) + (lo_ >> 1) + (other.lo_ >> 1)) >> 63;
+    hi_ += other.hi_ + carry;
+    lo_ += other.lo_;
+    return *this;
+  }
+
+  uint64_t hi_;
+  uint64_t lo_;
+};
+
+uint128_t operator*(const uint128_t& left, const uint128_t& right) {
+  uint128_t r;
+  ExtendAndMultiplyUint(left.lo_, right.lo_, &r.hi_, &r.lo_);
+  r.hi_ += (left.hi_ * right.lo_) + (left.lo_ * right.hi_);
+  return r;
+}
 #endif
+
+// Multiplies two N * 64 bit unsigned integer types, represented by a uint64_t
+// array into a same sized output. Overflow in multiplication is considered UB

Review comment:
       What does UB mean?

##########
File path: cpp/src/arrow/util/basic_decimal.cc
##########
@@ -254,67 +252,126 @@ BasicDecimal128& BasicDecimal128::operator>>=(uint32_t bits) {
 
 namespace {
 
-// TODO: Remove this guard once it's used by BasicDecimal256
-#ifndef ARROW_USE_NATIVE_INT128
-// This method losslessly multiplies x and y into a 128 bit unsigned integer
-// whose high bits will be stored in hi and low bits in lo.
-void ExtendAndMultiplyUint64(uint64_t x, uint64_t y, uint64_t* hi, uint64_t* lo) {
-#ifdef ARROW_USE_NATIVE_INT128
-  const __uint128_t r = static_cast<__uint128_t>(x) * y;
-  *lo = r & kInt64Mask;
-  *hi = r >> 64;
-#else
-  // If we can't use a native fallback, perform multiplication
-  // by splitting up x and y into 32 bit high/low bit components,
+// Multiply two N bit word components into a 2*N bit result, with high bits
+// stored in hi and low bits in lo.
+template <typename Word>
+inline void ExtendAndMultiplyUint(Word x, Word y, Word* hi, Word* lo) {
+  // Perform multiplication on two N bit words x and y into a 2*N bit result
+  // by splitting up x and y into N/2 bit high/low bit components,
   // allowing us to represent the multiplication as
-  // x * y = x_lo * y_lo + x_hi * y_lo * 2^32 + y_hi * x_lo * 2^32
-  // + x_hi * y_hi * 2^64.
+  // x * y = x_lo * y_lo + x_hi * y_lo * 2^N/2 + y_hi * x_lo * 2^N/2
+  // + x_hi * y_hi * 2^N
   //
-  // Now, consider the final output as lo_lo || lo_hi || hi_lo || hi_hi.
+  // Now, consider the final output as lo_lo || lo_hi || hi_lo || hi_hi
   // Therefore,
   // lo_lo is (x_lo * y_lo)_lo,
   // lo_hi is ((x_lo * y_lo)_hi + (x_hi * y_lo)_lo + (x_lo * y_hi)_lo)_lo,
   // hi_lo is ((x_hi * y_hi)_lo + (x_hi * y_lo)_hi + (x_lo * y_hi)_hi)_hi,
   // hi_hi is (x_hi * y_hi)_hi
-  const uint64_t x_lo = x & kIntMask;
-  const uint64_t y_lo = y & kIntMask;
-  const uint64_t x_hi = x >> 32;
-  const uint64_t y_hi = y >> 32;
+  constexpr Word kHighBitShift = sizeof(Word) * 4;
+  constexpr Word kLowBitMask = (static_cast<Word>(1) << kHighBitShift) - 1;
 
-  const uint64_t t = x_lo * y_lo;
-  const uint64_t t_lo = t & kIntMask;
-  const uint64_t t_hi = t >> 32;
+  const Word x_lo = x & kLowBitMask;
+  const Word y_lo = y & kLowBitMask;
+  const Word x_hi = x >> kHighBitShift;
+  const Word y_hi = y >> kHighBitShift;
 
-  const uint64_t u = x_hi * y_lo + t_hi;
-  const uint64_t u_lo = u & kIntMask;
-  const uint64_t u_hi = u >> 32;
+  const Word t = x_lo * y_lo;
+  const Word t_lo = t & kLowBitMask;
+  const Word t_hi = t >> kHighBitShift;
 
-  const uint64_t v = x_lo * y_hi + u_lo;
-  const uint64_t v_hi = v >> 32;
+  const Word u = x_hi * y_lo + t_hi;
+  const Word u_lo = u & kLowBitMask;
+  const Word u_hi = u >> kHighBitShift;
+
+  const Word v = x_lo * y_hi + u_lo;
+  const Word v_hi = v >> kHighBitShift;
 
   *hi = x_hi * y_hi + u_hi + v_hi;
-  *lo = (v << 32) | t_lo;
-#endif
+  *lo = (v << kHighBitShift) + t_lo;
 }
-#endif
 
-void MultiplyUint128(uint64_t x_hi, uint64_t x_lo, uint64_t y_hi, uint64_t y_lo,
-                     uint64_t* hi, uint64_t* lo) {
+// Convenience wrapper type over 128 bit unsigned integers. We opt not to
+// replace the uint128_t type in int128_internal.h because it would require
+// significantly more implementation work to be done. This class merely
+// provides the minimum necessary set of functions to perform 128+ bit
+// multiplication operations when there may or may not be native support.
 #ifdef ARROW_USE_NATIVE_INT128
-  const __uint128_t x = (static_cast<__uint128_t>(x_hi) << 64) | x_lo;
-  const __uint128_t y = (static_cast<__uint128_t>(y_hi) << 64) | y_lo;
-  const __uint128_t r = x * y;
-  *lo = r & kInt64Mask;
-  *hi = r >> 64;
+struct uint128_t {
+  uint128_t() {}
+  uint128_t(uint64_t hi, uint64_t lo) : val_((static_cast<__uint128_t>(hi) << 64) | lo) {}
+  uint128_t(const BasicDecimal128& decimal) {
+    val_ = (static_cast<__uint128_t>(decimal.high_bits()) << 64) | decimal.low_bits();
+  }
+
+  uint64_t hi() { return val_ >> 64; }
+  uint64_t lo() { return val_ & kInt64Mask; }
+
+  uint128_t& operator+=(const uint128_t& other) {
+    val_ += other.val_;
+    return *this;
+  }
+
+  __uint128_t val_;
+};
+
+uint128_t operator*(const uint128_t& left, const uint128_t& right) {

Review comment:
       Please try defining operator*= instead of operator*. Maybe this can help the compiler generate more efficient code.

##########
File path: cpp/src/arrow/util/basic_decimal.cc
##########
@@ -254,67 +252,126 @@ BasicDecimal128& BasicDecimal128::operator>>=(uint32_t bits) {
 
 namespace {
 
-// TODO: Remove this guard once it's used by BasicDecimal256
-#ifndef ARROW_USE_NATIVE_INT128
-// This method losslessly multiplies x and y into a 128 bit unsigned integer
-// whose high bits will be stored in hi and low bits in lo.
-void ExtendAndMultiplyUint64(uint64_t x, uint64_t y, uint64_t* hi, uint64_t* lo) {
-#ifdef ARROW_USE_NATIVE_INT128
-  const __uint128_t r = static_cast<__uint128_t>(x) * y;
-  *lo = r & kInt64Mask;
-  *hi = r >> 64;
-#else
-  // If we can't use a native fallback, perform multiplication
-  // by splitting up x and y into 32 bit high/low bit components,
+// Multiply two N bit word components into a 2*N bit result, with high bits
+// stored in hi and low bits in lo.
+template <typename Word>
+inline void ExtendAndMultiplyUint(Word x, Word y, Word* hi, Word* lo) {

Review comment:
       Now this method only needs to handle uint64 inputs, and it only needs to be defined in the #else block, right?




----------------------------------------------------------------
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] Luminarys commented on pull request #8344: Add BasicDecimal256 Multiplication Support (PR for decimal256 branch, not master)

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


   I've looked through the CI failures, it seems there are a few kinds:
   1. aws connector failure (I think this isn't our issue)
   2. a python lint error (this should be fixed, but maybe not in this PR)
   3. Arrow Gandiva compile error (same as above)
   4. Some issue around the new constructor I defined (I'll investigate this)
   5. MinGW SDK not found (I think this isn't our issue)
   


----------------------------------------------------------------
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 merged pull request #8344: Add BasicDecimal256 Multiplication Support (PR for decimal256 branch, not master)

Posted by GitBox <gi...@apache.org>.
emkornfield merged pull request #8344:
URL: https://github.com/apache/arrow/pull/8344


   


----------------------------------------------------------------
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] Luminarys commented on a change in pull request #8344: Add BasicDecimal256 Multiplication Support (PR for decimal256 branch, not master)

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



##########
File path: cpp/src/arrow/util/basic_decimal.cc
##########
@@ -254,69 +252,148 @@ BasicDecimal128& BasicDecimal128::operator>>=(uint32_t bits) {
 
 namespace {
 
-// TODO: Remove this guard once it's used by BasicDecimal256
-#ifndef ARROW_USE_NATIVE_INT128
-// This method losslessly multiplies x and y into a 128 bit unsigned integer
-// whose high bits will be stored in hi and low bits in lo.
-void ExtendAndMultiplyUint64(uint64_t x, uint64_t y, uint64_t* hi, uint64_t* lo) {
-#ifdef ARROW_USE_NATIVE_INT128
-  const __uint128_t r = static_cast<__uint128_t>(x) * y;
-  *lo = r & kInt64Mask;
-  *hi = r >> 64;
-#else
-  // If we can't use a native fallback, perform multiplication
-  // by splitting up x and y into 32 bit high/low bit components,
+// Multiply two N bit word components into a 2*N bit result, with high bits
+// stored in hi and low bits in lo.
+template <typename Word>
+void ExtendAndMultiplyUint(Word x, Word y, Word* hi, Word* lo) {
+  // Perform multiplication on two N bit words x and y into a 2*N bit result
+  // by splitting up x and y into N/2 bit high/low bit components,
   // allowing us to represent the multiplication as
-  // x * y = x_lo * y_lo + x_hi * y_lo * 2^32 + y_hi * x_lo * 2^32
-  // + x_hi * y_hi * 2^64.
+  // x * y = x_lo * y_lo + x_hi * y_lo * 2^N/2 + y_hi * x_lo * 2^N/2
+  // + x_hi * y_hi * 2^N
   //
-  // Now, consider the final output as lo_lo || lo_hi || hi_lo || hi_hi.
+  // Now, consider the final output as lo_lo || lo_hi || hi_lo || hi_hi
   // Therefore,
   // lo_lo is (x_lo * y_lo)_lo,
   // lo_hi is ((x_lo * y_lo)_hi + (x_hi * y_lo)_lo + (x_lo * y_hi)_lo)_lo,
   // hi_lo is ((x_hi * y_hi)_lo + (x_hi * y_lo)_hi + (x_lo * y_hi)_hi)_hi,
   // hi_hi is (x_hi * y_hi)_hi
-  const uint64_t x_lo = x & kIntMask;
-  const uint64_t y_lo = y & kIntMask;
-  const uint64_t x_hi = x >> 32;
-  const uint64_t y_hi = y >> 32;
+  constexpr Word kHighBitShift = sizeof(Word) * 4;
+  constexpr Word kLowBitMask = (static_cast<Word>(1) << kHighBitShift) - 1;
 
-  const uint64_t t = x_lo * y_lo;
-  const uint64_t t_lo = t & kIntMask;
-  const uint64_t t_hi = t >> 32;
+  const Word x_lo = x & kLowBitMask;
+  const Word y_lo = y & kLowBitMask;
+  const Word x_hi = x >> kHighBitShift;
+  const Word y_hi = y >> kHighBitShift;
 
-  const uint64_t u = x_hi * y_lo + t_hi;
-  const uint64_t u_lo = u & kIntMask;
-  const uint64_t u_hi = u >> 32;
+  const Word t = x_lo * y_lo;
+  const Word t_lo = t & kLowBitMask;
+  const Word t_hi = t >> kHighBitShift;
 
-  const uint64_t v = x_lo * y_hi + u_lo;
-  const uint64_t v_hi = v >> 32;
+  const Word u = x_hi * y_lo + t_hi;
+  const Word u_lo = u & kLowBitMask;
+  const Word u_hi = u >> kHighBitShift;
+
+  const Word v = x_lo * y_hi + u_lo;
+  const Word v_hi = v >> kHighBitShift;
 
   *hi = x_hi * y_hi + u_hi + v_hi;
-  *lo = (v << 32) | t_lo;
-#endif
+  *lo = (v << kHighBitShift) + t_lo;
+}
+
+// Convenience wrapper type over 128 bit unsigned integers
+#ifdef ARROW_USE_NATIVE_INT128
+struct uint128_t {

Review comment:
       My main concern with moving and replacing the existing alias is that we'll now have to ensure that the type functions exactly how the native/boost type do, which is likely going to require a lot of unnecessary effort to provide extra functions. I'd rather keep this localized and lightweight until there's an actual need to reuse this (since it really is just a convenience wrapper here).

##########
File path: cpp/src/arrow/util/basic_decimal.cc
##########
@@ -254,69 +252,148 @@ BasicDecimal128& BasicDecimal128::operator>>=(uint32_t bits) {
 
 namespace {
 
-// TODO: Remove this guard once it's used by BasicDecimal256
-#ifndef ARROW_USE_NATIVE_INT128
-// This method losslessly multiplies x and y into a 128 bit unsigned integer
-// whose high bits will be stored in hi and low bits in lo.
-void ExtendAndMultiplyUint64(uint64_t x, uint64_t y, uint64_t* hi, uint64_t* lo) {
-#ifdef ARROW_USE_NATIVE_INT128
-  const __uint128_t r = static_cast<__uint128_t>(x) * y;
-  *lo = r & kInt64Mask;
-  *hi = r >> 64;
-#else
-  // If we can't use a native fallback, perform multiplication
-  // by splitting up x and y into 32 bit high/low bit components,
+// Multiply two N bit word components into a 2*N bit result, with high bits
+// stored in hi and low bits in lo.
+template <typename Word>
+void ExtendAndMultiplyUint(Word x, Word y, Word* hi, Word* lo) {
+  // Perform multiplication on two N bit words x and y into a 2*N bit result
+  // by splitting up x and y into N/2 bit high/low bit components,
   // allowing us to represent the multiplication as
-  // x * y = x_lo * y_lo + x_hi * y_lo * 2^32 + y_hi * x_lo * 2^32
-  // + x_hi * y_hi * 2^64.
+  // x * y = x_lo * y_lo + x_hi * y_lo * 2^N/2 + y_hi * x_lo * 2^N/2
+  // + x_hi * y_hi * 2^N
   //
-  // Now, consider the final output as lo_lo || lo_hi || hi_lo || hi_hi.
+  // Now, consider the final output as lo_lo || lo_hi || hi_lo || hi_hi
   // Therefore,
   // lo_lo is (x_lo * y_lo)_lo,
   // lo_hi is ((x_lo * y_lo)_hi + (x_hi * y_lo)_lo + (x_lo * y_hi)_lo)_lo,
   // hi_lo is ((x_hi * y_hi)_lo + (x_hi * y_lo)_hi + (x_lo * y_hi)_hi)_hi,
   // hi_hi is (x_hi * y_hi)_hi
-  const uint64_t x_lo = x & kIntMask;
-  const uint64_t y_lo = y & kIntMask;
-  const uint64_t x_hi = x >> 32;
-  const uint64_t y_hi = y >> 32;
+  constexpr Word kHighBitShift = sizeof(Word) * 4;
+  constexpr Word kLowBitMask = (static_cast<Word>(1) << kHighBitShift) - 1;
 
-  const uint64_t t = x_lo * y_lo;
-  const uint64_t t_lo = t & kIntMask;
-  const uint64_t t_hi = t >> 32;
+  const Word x_lo = x & kLowBitMask;
+  const Word y_lo = y & kLowBitMask;
+  const Word x_hi = x >> kHighBitShift;
+  const Word y_hi = y >> kHighBitShift;
 
-  const uint64_t u = x_hi * y_lo + t_hi;
-  const uint64_t u_lo = u & kIntMask;
-  const uint64_t u_hi = u >> 32;
+  const Word t = x_lo * y_lo;
+  const Word t_lo = t & kLowBitMask;
+  const Word t_hi = t >> kHighBitShift;
 
-  const uint64_t v = x_lo * y_hi + u_lo;
-  const uint64_t v_hi = v >> 32;
+  const Word u = x_hi * y_lo + t_hi;
+  const Word u_lo = u & kLowBitMask;
+  const Word u_hi = u >> kHighBitShift;
+
+  const Word v = x_lo * y_hi + u_lo;
+  const Word v_hi = v >> kHighBitShift;
 
   *hi = x_hi * y_hi + u_hi + v_hi;
-  *lo = (v << 32) | t_lo;
-#endif
+  *lo = (v << kHighBitShift) + t_lo;
+}
+
+// Convenience wrapper type over 128 bit unsigned integers
+#ifdef ARROW_USE_NATIVE_INT128
+struct uint128_t {
+  uint128_t() {}
+  uint128_t(uint64_t hi, uint64_t lo) : val_((static_cast<__uint128_t>(hi) << 64) | lo) {}
+  uint128_t(const BasicDecimal128& decimal) {
+    val_ = (static_cast<__uint128_t>(decimal.high_bits()) << 64) | decimal.low_bits();
+  }
+
+  uint64_t hi() { return val_ >> 64; }
+  uint64_t lo() { return val_ & kInt64Mask; }
+
+  uint128_t& operator+=(const uint128_t& other) {
+    val_ += other.val_;
+    return *this;
+  }
+
+  __uint128_t val_;
+};
+
+uint128_t operator*(const uint128_t& left, const uint128_t& right) {
+  uint128_t r;
+  r.val_ = left.val_ * right.val_;
+  return r;
+}
+#else
+struct uint128_t {
+  uint128_t() {}
+  uint128_t(uint64_t hi, uint64_t lo) : hi_(hi), lo_(lo) {}
+  uint128_t(const BasicDecimal128& decimal) {
+    hi_ = decimal.high_bits();
+    lo_ = decimal.low_bits();
+  }
+
+  uint64_t hi() const { return hi_; }
+  uint64_t lo() const { return lo_; }
+
+  uint128_t& operator+=(const uint128_t& other) {
+    // To deduce the carry bit, we perform "65 bit" addition on the low bits and
+    // seeing if the resulting high bit is 1. This is accomplished by shifting the
+    // low bits to the right by 1 (chopping off the lowest bit), then adding 1 if the
+    // result of adding the two chopped bits would have produced a carry.
+    uint64_t carry = (((lo_ & other.lo_) & 1) + (lo_ >> 1) + (other.lo_ >> 1)) >> 63;
+    hi_ += other.hi_ + carry;
+    lo_ += other.lo_;
+    return *this;
+  }
+
+  uint64_t hi_;
+  uint64_t lo_;
+};
+
+uint128_t operator*(const uint128_t& left, const uint128_t& right) {
+  uint128_t r;
+  ExtendAndMultiplyUint(left.lo_, right.lo_, &r.hi_, &r.lo_);
+  r.hi_ += (left.hi_ * right.lo_) + (left.lo_ * right.hi_);
+  return r;
 }
 #endif
 
-void MultiplyUint128(uint64_t x_hi, uint64_t x_lo, uint64_t y_hi, uint64_t y_lo,
-                     uint64_t* hi, uint64_t* lo) {
+void ExtendAndMultiplyUint128(uint128_t x, uint128_t y, uint128_t* hi, uint128_t* lo) {

Review comment:
       This method is now removed.

##########
File path: cpp/src/arrow/util/basic_decimal.cc
##########
@@ -254,69 +252,148 @@ BasicDecimal128& BasicDecimal128::operator>>=(uint32_t bits) {
 
 namespace {
 
-// TODO: Remove this guard once it's used by BasicDecimal256
-#ifndef ARROW_USE_NATIVE_INT128
-// This method losslessly multiplies x and y into a 128 bit unsigned integer
-// whose high bits will be stored in hi and low bits in lo.
-void ExtendAndMultiplyUint64(uint64_t x, uint64_t y, uint64_t* hi, uint64_t* lo) {
-#ifdef ARROW_USE_NATIVE_INT128
-  const __uint128_t r = static_cast<__uint128_t>(x) * y;
-  *lo = r & kInt64Mask;
-  *hi = r >> 64;
-#else
-  // If we can't use a native fallback, perform multiplication
-  // by splitting up x and y into 32 bit high/low bit components,
+// Multiply two N bit word components into a 2*N bit result, with high bits
+// stored in hi and low bits in lo.
+template <typename Word>
+void ExtendAndMultiplyUint(Word x, Word y, Word* hi, Word* lo) {

Review comment:
       Done. This saves a lot of code, though does take 60 ns for multiplication as opposed to 20 ns prior.




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