You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by ze...@apache.org on 2022/05/17 14:59:00 UTC

[arrow] branch master updated: ARROW-16552: [Go] Improve decimal128 utilities

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

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


The following commit(s) were added to refs/heads/master by this push:
     new c4371ab0c7 ARROW-16552: [Go] Improve decimal128 utilities
c4371ab0c7 is described below

commit c4371ab0c75c711af9e91fa297bad51c4cc1271d
Author: Matthew Topol <mt...@factset.com>
AuthorDate: Tue May 17 10:58:52 2022 -0400

    ARROW-16552: [Go] Improve decimal128 utilities
    
    Adding new utilities for decimal128.Num for rescaling and for converting to and from float32/64
    
    Closes #13134 from zeroshade/arrow-16552-decimals
    
    Authored-by: Matthew Topol <mt...@factset.com>
    Signed-off-by: Matthew Topol <mt...@factset.com>
---
 go/arrow/decimal128/decimal128.go      | 332 ++++++++++++++++++++++++++++++++-
 go/arrow/decimal128/decimal128_test.go | 244 ++++++++++++++++++++++++
 2 files changed, 573 insertions(+), 3 deletions(-)

diff --git a/go/arrow/decimal128/decimal128.go b/go/arrow/decimal128/decimal128.go
index 2423784fce..14ad42ea5d 100644
--- a/go/arrow/decimal128/decimal128.go
+++ b/go/arrow/decimal128/decimal128.go
@@ -17,7 +17,12 @@
 package decimal128
 
 import (
+	"errors"
+	"fmt"
+	"math"
 	"math/big"
+
+	"github.com/apache/arrow/go/v9/arrow/internal/debug"
 )
 
 var (
@@ -81,12 +86,13 @@ func FromBigInt(v *big.Int) (n Num) {
 		n.hi = int64(b[1])
 	}
 	if v.Sign() < 0 {
-		return n.negated()
+		return n.Negate()
 	}
 	return
 }
 
-func (n Num) negated() Num {
+// Negate returns a copy of this Decimal128 value but with the sign negated
+func (n Num) Negate() Num {
 	n.lo = ^n.lo + 1
 	n.hi = ^n.hi
 	if n.lo == 0 {
@@ -95,6 +101,116 @@ func (n Num) negated() Num {
 	return n
 }
 
+func fromPositiveFloat32(v float32, prec, scale int32) (Num, error) {
+	var pscale float32
+	if scale >= -38 && scale <= 38 {
+		pscale = float32PowersOfTen[scale+38]
+	} else {
+		pscale = float32(math.Pow10(int(scale)))
+	}
+
+	v *= pscale
+	v = float32(math.RoundToEven(float64(v)))
+	maxabs := float32PowersOfTen[prec+38]
+	if v <= -maxabs || v >= maxabs {
+		return Num{}, fmt.Errorf("cannot convert %f to decimal128(precision=%d, scale=%d): overflow", v, prec, scale)
+	}
+
+	hi := float32(math.Floor(math.Ldexp(float64(v), -64)))
+	low := v - float32(math.Ldexp(float64(hi), 64))
+	return Num{hi: int64(hi), lo: uint64(low)}, nil
+}
+
+func fromPositiveFloat64(v float64, prec, scale int32) (Num, error) {
+	var pscale float64
+	if scale >= -38 && scale <= 38 {
+		pscale = float64PowersOfTen[scale+38]
+	} else {
+		pscale = math.Pow10(int(scale))
+	}
+
+	v *= pscale
+	v = math.RoundToEven(v)
+	maxabs := float64PowersOfTen[prec+38]
+	if v <= -maxabs || v >= maxabs {
+		return Num{}, fmt.Errorf("cannot convert %f to decimal128(precision=%d, scale=%d): overflow", v, prec, scale)
+	}
+
+	hi := math.Floor(math.Ldexp(float64(v), -64))
+	low := v - math.Ldexp(hi, 64)
+	return Num{hi: int64(hi), lo: uint64(low)}, nil
+}
+
+// FromFloat32 returns a new decimal128.Num constructed from the given float32
+// value using the provided precision and scale. Will return an error if the
+// value cannot be accurately represented with the desired precision and scale.
+func FromFloat32(v float32, prec, scale int32) (Num, error) {
+	if v < 0 {
+		dec, err := fromPositiveFloat32(-v, prec, scale)
+		if err != nil {
+			return dec, err
+		}
+		return dec.Negate(), nil
+	}
+	return fromPositiveFloat32(v, prec, scale)
+}
+
+// FromFloat64 returns a new decimal128.Num constructed from the given float64
+// value using the provided precision and scale. Will return an error if the
+// value cannot be accurately represented with the desired precision and scale.
+func FromFloat64(v float64, prec, scale int32) (Num, error) {
+	if v < 0 {
+		dec, err := fromPositiveFloat64(-v, prec, scale)
+		if err != nil {
+			return dec, err
+		}
+		return dec.Negate(), nil
+	}
+	return fromPositiveFloat64(v, prec, scale)
+}
+
+func (n Num) tofloat32Positive(scale int32) float32 {
+	const twoTo64 float32 = 1.8446744e+19
+	x := float32(n.hi) * twoTo64
+	x += float32(n.lo)
+	if scale >= -38 && scale <= 38 {
+		x *= float32PowersOfTen[-scale+38]
+	} else {
+		x *= float32(math.Pow10(-int(scale)))
+	}
+	return x
+}
+
+// ToFloat32 returns a float32 value representative of this decimal128.Num,
+// but with the given scale.
+func (n Num) ToFloat32(scale int32) float32 {
+	if n.hi < 0 {
+		return -n.Negate().tofloat32Positive(scale)
+	}
+	return n.tofloat32Positive(scale)
+}
+
+func (n Num) tofloat64Positive(scale int32) float64 {
+	const twoTo64 float64 = 1.8446744073709552e+19
+	x := float64(n.hi) * twoTo64
+	x += float64(n.lo)
+	if scale >= -38 && scale <= 38 {
+		x *= float64PowersOfTen[-scale+38]
+	} else {
+		x *= math.Pow10(-int(scale))
+	}
+	return x
+}
+
+// ToFloat64 returns a float64 value representative of this decimal128.Num,
+// but with the given scale.
+func (n Num) ToFloat64(scale int32) float64 {
+	if n.hi < 0 {
+		return -n.Negate().tofloat64Positive(scale)
+	}
+	return n.tofloat64Positive(scale)
+}
+
 // LowBits returns the low bits of the two's complement representation of the number.
 func (n Num) LowBits() uint64 { return n.lo }
 
@@ -122,8 +238,218 @@ func toBigIntPositive(n Num) *big.Int {
 // in the words and negating ends up being >2x faster
 func (n Num) BigInt() *big.Int {
 	if n.Sign() < 0 {
-		b := toBigIntPositive(n.negated())
+		b := toBigIntPositive(n.Negate())
 		return b.Neg(b)
 	}
 	return toBigIntPositive(n)
 }
+
+// Less returns true if the value represented by n is < other
+func (n Num) Less(other Num) bool {
+	return n.hi < other.hi || (n.hi == other.hi && n.lo < other.lo)
+}
+
+// IncreaseScaleBy returns a new decimal128.Num with the value scaled up by
+// the desired amount. Must be 0 <= increase <= 38. Any data loss from scaling
+// is ignored. If you wish to prevent data loss, use Rescale which will
+// return an error if data loss is detected.
+func (n Num) IncreaseScaleBy(increase int32) Num {
+	debug.Assert(increase >= 0, "invalid increase scale for decimal128")
+	debug.Assert(increase <= 38, "invalid increase scale for decimal128")
+
+	v := scaleMultipliers[increase].BigInt()
+	return FromBigInt(v.Mul(n.BigInt(), v))
+}
+
+// ReduceScaleBy returns a new decimal128.Num with the value scaled down by
+// the desired amount and, if 'round' is true, the value will be rounded
+// accordingly. Assumes 0 <= reduce <= 38. Any data loss from scaling
+// is ignored. If you wish to prevent data loss, use Rescale which will
+// return an error if data loss is detected.
+func (n Num) ReduceScaleBy(reduce int32, round bool) Num {
+	debug.Assert(reduce >= 0, "invalid reduce scale for decimal128")
+	debug.Assert(reduce <= 38, "invalid reduce scale for decimal128")
+
+	if reduce == 0 {
+		return n
+	}
+
+	divisor := scaleMultipliers[reduce].BigInt()
+	result, remainder := divisor.QuoRem(n.BigInt(), divisor, (&big.Int{}))
+	if round {
+		divisorHalf := scaleMultipliersHalf[reduce]
+		if remainder.Abs(remainder).Cmp(divisorHalf.BigInt()) != -1 {
+			result.Add(result, big.NewInt(int64(n.Sign())))
+		}
+	}
+	return FromBigInt(result)
+}
+
+func (n Num) rescaleWouldCauseDataLoss(deltaScale int32, multiplier Num) (out Num, loss bool) {
+	var (
+		value, result, remainder *big.Int
+	)
+	value = n.BigInt()
+	if deltaScale < 0 {
+		debug.Assert(multiplier.lo != 0 || multiplier.hi != 0, "multiplier needs to not be zero")
+		result, remainder = (&big.Int{}).QuoRem(value, multiplier.BigInt(), (&big.Int{}))
+		return FromBigInt(result), remainder.Cmp(big.NewInt(0)) != 0
+	}
+
+	result = (&big.Int{}).Mul(value, multiplier.BigInt())
+	out = FromBigInt(result)
+	cmp := result.Cmp(value)
+	if n.Sign() < 0 {
+		loss = cmp == 1
+	} else {
+		loss = cmp == -1
+	}
+	return
+}
+
+// Rescale returns a new decimal128.Num with the value updated assuming
+// the current value is scaled to originalScale with the new value scaled
+// to newScale. If rescaling this way would cause data loss, an error is
+// returned instead.
+func (n Num) Rescale(originalScale, newScale int32) (out Num, err error) {
+	if originalScale == newScale {
+		return n, nil
+	}
+
+	deltaScale := newScale - originalScale
+	absDeltaScale := int32(math.Abs(float64(deltaScale)))
+
+	multiplier := scaleMultipliers[absDeltaScale]
+	var wouldHaveLoss bool
+	out, wouldHaveLoss = n.rescaleWouldCauseDataLoss(deltaScale, multiplier)
+	if wouldHaveLoss {
+		err = errors.New("rescale data loss")
+	}
+	return
+}
+
+// Abs returns a new decimal128.Num that contains the absolute value of n
+func (n Num) Abs() Num {
+	switch n.Sign() {
+	case -1:
+		return n.Negate()
+	}
+	return n
+}
+
+// FitsInPrecision returns true or false if the value currently held by
+// n would fit within precision (0 < prec <= 38) without losing any data.
+func (n Num) FitsInPrecision(prec int32) bool {
+	debug.Assert(prec > 0, "precision must be > 0")
+	debug.Assert(prec <= 38, "precision must be <= 38")
+	return n.Abs().Less(scaleMultipliers[prec])
+}
+
+var (
+	scaleMultipliers = [...]Num{
+		FromU64(1),
+		FromU64(10),
+		FromU64(100),
+		FromU64(1000),
+		FromU64(10000),
+		FromU64(100000),
+		FromU64(1000000),
+		FromU64(10000000),
+		FromU64(100000000),
+		FromU64(1000000000),
+		FromU64(10000000000),
+		FromU64(100000000000),
+		FromU64(1000000000000),
+		FromU64(10000000000000),
+		FromU64(100000000000000),
+		FromU64(1000000000000000),
+		FromU64(10000000000000000),
+		FromU64(100000000000000000),
+		FromU64(1000000000000000000),
+		FromU64(10000000000000000000),
+		New(0, 10000000000000000000),
+		New(5, 7766279631452241920),
+		New(54, 3875820019684212736),
+		New(542, 1864712049423024128),
+		New(5421, 200376420520689664),
+		New(54210, 2003764205206896640),
+		New(542101, 1590897978359414784),
+		New(5421010, 15908979783594147840),
+		New(54210108, 11515845246265065472),
+		New(542101086, 4477988020393345024),
+		New(5421010862, 7886392056514347008),
+		New(54210108624, 5076944270305263616),
+		New(542101086242, 13875954555633532928),
+		New(5421010862427, 9632337040368467968),
+		New(54210108624275, 4089650035136921600),
+		New(542101086242752, 4003012203950112768),
+		New(5421010862427522, 3136633892082024448),
+		New(54210108624275221, 12919594847110692864),
+		New(542101086242752217, 68739955140067328),
+		New(5421010862427522170, 687399551400673280),
+	}
+
+	scaleMultipliersHalf = [...]Num{
+		FromU64(0),
+		FromU64(5),
+		FromU64(50),
+		FromU64(500),
+		FromU64(5000),
+		FromU64(50000),
+		FromU64(500000),
+		FromU64(5000000),
+		FromU64(50000000),
+		FromU64(500000000),
+		FromU64(5000000000),
+		FromU64(50000000000),
+		FromU64(500000000000),
+		FromU64(5000000000000),
+		FromU64(50000000000000),
+		FromU64(500000000000000),
+		FromU64(5000000000000000),
+		FromU64(50000000000000000),
+		FromU64(500000000000000000),
+		FromU64(5000000000000000000),
+		New(2, 13106511852580896768),
+		New(27, 1937910009842106368),
+		New(271, 932356024711512064),
+		New(2710, 9323560247115120640),
+		New(27105, 1001882102603448320),
+		New(271050, 10018821026034483200),
+		New(2710505, 7954489891797073920),
+		New(27105054, 5757922623132532736),
+		New(271050543, 2238994010196672512),
+		New(2710505431, 3943196028257173504),
+		New(27105054312, 2538472135152631808),
+		New(271050543121, 6937977277816766464),
+		New(2710505431213, 14039540557039009792),
+		New(27105054312137, 11268197054423236608),
+		New(271050543121376, 2001506101975056384),
+		New(2710505431213761, 1568316946041012224),
+		New(27105054312137610, 15683169460410122240),
+		New(271050543121376108, 9257742014424809472),
+		New(2710505431213761085, 343699775700336640),
+	}
+
+	float32PowersOfTen = [...]float32{
+		1e-38, 1e-37, 1e-36, 1e-35, 1e-34, 1e-33, 1e-32, 1e-31, 1e-30, 1e-29,
+		1e-28, 1e-27, 1e-26, 1e-25, 1e-24, 1e-23, 1e-22, 1e-21, 1e-20, 1e-19,
+		1e-18, 1e-17, 1e-16, 1e-15, 1e-14, 1e-13, 1e-12, 1e-11, 1e-10, 1e-9,
+		1e-8, 1e-7, 1e-6, 1e-5, 1e-4, 1e-3, 1e-2, 1e-1, 1e0, 1e1,
+		1e2, 1e3, 1e4, 1e5, 1e6, 1e7, 1e8, 1e9, 1e10, 1e11,
+		1e12, 1e13, 1e14, 1e15, 1e16, 1e17, 1e18, 1e19, 1e20, 1e21,
+		1e22, 1e23, 1e24, 1e25, 1e26, 1e27, 1e28, 1e29, 1e30, 1e31,
+		1e32, 1e33, 1e34, 1e35, 1e36, 1e37, 1e38,
+	}
+
+	float64PowersOfTen = [...]float64{
+		1e-38, 1e-37, 1e-36, 1e-35, 1e-34, 1e-33, 1e-32, 1e-31, 1e-30, 1e-29,
+		1e-28, 1e-27, 1e-26, 1e-25, 1e-24, 1e-23, 1e-22, 1e-21, 1e-20, 1e-19,
+		1e-18, 1e-17, 1e-16, 1e-15, 1e-14, 1e-13, 1e-12, 1e-11, 1e-10, 1e-9,
+		1e-8, 1e-7, 1e-6, 1e-5, 1e-4, 1e-3, 1e-2, 1e-1, 1e0, 1e1,
+		1e2, 1e3, 1e4, 1e5, 1e6, 1e7, 1e8, 1e9, 1e10, 1e11,
+		1e12, 1e13, 1e14, 1e15, 1e16, 1e17, 1e18, 1e19, 1e20, 1e21,
+		1e22, 1e23, 1e24, 1e25, 1e26, 1e27, 1e28, 1e29, 1e30, 1e31,
+		1e32, 1e33, 1e34, 1e35, 1e36, 1e37, 1e38,
+	}
+)
diff --git a/go/arrow/decimal128/decimal128_test.go b/go/arrow/decimal128/decimal128_test.go
index 5e61549fe9..e986e95fe8 100644
--- a/go/arrow/decimal128/decimal128_test.go
+++ b/go/arrow/decimal128/decimal128_test.go
@@ -148,3 +148,247 @@ func TestDecimalToBigInt(t *testing.T) {
 		})
 	}
 }
+
+func ulps64(actual, expected float64) int64 {
+	ulp := math.Nextafter(actual, math.Inf(1)) - actual
+	return int64(math.Abs((expected - actual) / ulp))
+}
+
+func ulps32(actual, expected float32) int64 {
+	ulp := math.Nextafter32(actual, float32(math.Inf(1))) - actual
+	return int64(math.Abs(float64((expected - actual) / ulp)))
+}
+
+func assertFloat32Approx(t *testing.T, x, y float32) bool {
+	const maxulps int64 = 4
+	ulps := ulps32(x, y)
+	return assert.LessOrEqualf(t, ulps, maxulps, "%f not equal to %f (%d ulps)", x, y, ulps)
+}
+
+func assertFloat64Approx(t *testing.T, x, y float64) bool {
+	const maxulps int64 = 4
+	ulps := ulps64(x, y)
+	return assert.LessOrEqualf(t, ulps, maxulps, "%f not equal to %f (%d ulps)", x, y, ulps)
+}
+
+func TestDecimalToReal(t *testing.T) {
+	tests := []struct {
+		decimalVal string
+		scale      int32
+		exp        float64
+	}{
+		{"0", 0, 0},
+		{"0", 10, 0.0},
+		{"0", -10, 0.0},
+		{"1", 0, 1.0},
+		{"12345", 0, 12345.0},
+		{"12345", 1, 1234.5},
+		// 2**62
+		{"4611686018427387904", 0, math.Pow(2, 62)},
+		// 2**63 + 2**62
+		{"13835058055282163712", 0, math.Pow(2, 63) + math.Pow(2, 62)},
+		// 2**64 + 2**62
+		{"23058430092136939520", 0, math.Pow(2, 64) + math.Pow(2, 62)},
+		// 10**38 - 2**103
+		{"99999989858795198174164788026374356992", 0, math.Pow10(38) - math.Pow(2, 103)},
+	}
+
+	t.Run("float32", func(t *testing.T) {
+		checkDecimalToFloat := func(t *testing.T, str string, v float32, scale int32) {
+			bi, _ := (&big.Int{}).SetString(str, 10)
+			dec := decimal128.FromBigInt(bi)
+			assert.Equalf(t, v, dec.ToFloat32(scale), "Decimal Val: %s, Scale: %d", str, scale)
+		}
+		for _, tt := range tests {
+			t.Run(tt.decimalVal, func(t *testing.T) {
+				checkDecimalToFloat(t, tt.decimalVal, float32(tt.exp), tt.scale)
+				if tt.decimalVal != "0" {
+					checkDecimalToFloat(t, "-"+tt.decimalVal, float32(-tt.exp), tt.scale)
+				}
+			})
+		}
+
+		t.Run("precision", func(t *testing.T) {
+			// 2**63 + 2**40 (exactly representable in a float's 24 bits of precision)
+			checkDecimalToFloat(t, "9223373136366403584", float32(9.223373e+18), 0)
+			checkDecimalToFloat(t, "-9223373136366403584", float32(-9.223373e+18), 0)
+			// 2**64 + 2**41 exactly representable in a float
+			checkDecimalToFloat(t, "18446746272732807168", float32(1.8446746e+19), 0)
+			checkDecimalToFloat(t, "-18446746272732807168", float32(-1.8446746e+19), 0)
+		})
+
+		t.Run("large values", func(t *testing.T) {
+			checkApproxDecimalToFloat := func(str string, v float32, scale int32) {
+				bi, _ := (&big.Int{}).SetString(str, 10)
+				dec := decimal128.FromBigInt(bi)
+				assertFloat32Approx(t, v, dec.ToFloat32(scale))
+			}
+			// exact comparisons would succeed on most platforms, but not all power-of-ten
+			// factors are exactly representable in binary floating point, so we'll use
+			// approx and ensure that the values are within 4 ULP (unit of least precision)
+			for scale := int32(-38); scale <= 38; scale++ {
+				checkApproxDecimalToFloat("1", float32(math.Pow10(-int(scale))), scale)
+				checkApproxDecimalToFloat("123", float32(123)*float32(math.Pow10(-int(scale))), scale)
+			}
+		})
+	})
+
+	t.Run("float64", func(t *testing.T) {
+		checkDecimalToFloat := func(t *testing.T, str string, v float64, scale int32) {
+			bi, _ := (&big.Int{}).SetString(str, 10)
+			dec := decimal128.FromBigInt(bi)
+			assert.Equalf(t, v, dec.ToFloat64(scale), "Decimal Val: %s, Scale: %d", str, scale)
+		}
+		for _, tt := range tests {
+			t.Run(tt.decimalVal, func(t *testing.T) {
+				checkDecimalToFloat(t, tt.decimalVal, tt.exp, tt.scale)
+				if tt.decimalVal != "0" {
+					checkDecimalToFloat(t, "-"+tt.decimalVal, -tt.exp, tt.scale)
+				}
+			})
+		}
+
+		t.Run("precision", func(t *testing.T) {
+			// 2**63 + 2**11 (exactly representable in float64's 53 bits of precision)
+			checkDecimalToFloat(t, "9223373136366403584", float64(9.223373136366404e+18), 0)
+			checkDecimalToFloat(t, "-9223373136366403584", float64(-9.223373136366404e+18), 0)
+
+			// 2**64 - 2**11 (exactly represntable in a float64)
+			checkDecimalToFloat(t, "18446746272732807168", float64(1.8446746272732807e+19), 0)
+			checkDecimalToFloat(t, "-18446746272732807168", float64(-1.8446746272732807e+19), 0)
+
+			// 2**64 + 2**11 (exactly representable in a float64)
+			checkDecimalToFloat(t, "18446744073709555712", float64(1.8446744073709556e+19), 0)
+			checkDecimalToFloat(t, "-18446744073709555712", float64(-1.8446744073709556e+19), 0)
+
+			// Almost 10**38 (minus 2**73)
+			checkDecimalToFloat(t, "99999999999999978859343891977453174784", 9.999999999999998e+37, 0)
+			checkDecimalToFloat(t, "-99999999999999978859343891977453174784", -9.999999999999998e+37, 0)
+			checkDecimalToFloat(t, "99999999999999978859343891977453174784", 9.999999999999998e+27, 10)
+			checkDecimalToFloat(t, "-99999999999999978859343891977453174784", -9.999999999999998e+27, 10)
+			checkDecimalToFloat(t, "99999999999999978859343891977453174784", 9.999999999999998e+47, -10)
+			checkDecimalToFloat(t, "-99999999999999978859343891977453174784", -9.999999999999998e+47, -10)
+		})
+
+		t.Run("large values", func(t *testing.T) {
+			checkApproxDecimalToFloat := func(str string, v float64, scale int32) {
+				bi, _ := (&big.Int{}).SetString(str, 10)
+				dec := decimal128.FromBigInt(bi)
+				assertFloat64Approx(t, v, dec.ToFloat64(scale))
+			}
+			// exact comparisons would succeed on most platforms, but not all power-of-ten
+			// factors are exactly representable in binary floating point, so we'll use
+			// approx and ensure that the values are within 4 ULP (unit of least precision)
+			for scale := int32(-308); scale <= 306; scale++ {
+				checkApproxDecimalToFloat("1", math.Pow10(-int(scale)), scale)
+				checkApproxDecimalToFloat("123", float64(123)*math.Pow10(-int(scale)), scale)
+			}
+		})
+	})
+}
+
+func TestDecimalFromFloat(t *testing.T) {
+	tests := []struct {
+		val              float64
+		precision, scale int32
+		expected         string
+	}{
+		{0, 1, 0, "0"},
+		{-0, 1, 0, "0"},
+		{0, 19, 4, "0.0000"},
+		{math.Copysign(0.0, -1), 19, 4, "0.0000"},
+		{123, 7, 4, "123.0000"},
+		{-123, 7, 4, "-123.0000"},
+		{456.78, 7, 4, "456.7800"},
+		{-456.78, 7, 4, "-456.7800"},
+		{456.784, 5, 2, "456.78"},
+		{-456.784, 5, 2, "-456.78"},
+		{456.786, 5, 2, "456.79"},
+		{-456.786, 5, 2, "-456.79"},
+		{999.99, 5, 2, "999.99"},
+		{-999.99, 5, 2, "-999.99"},
+		{123, 19, 0, "123"},
+		{-123, 19, 0, "-123"},
+		{123.4, 19, 0, "123"},
+		{-123.4, 19, 0, "-123"},
+		{123.6, 19, 0, "124"},
+		{-123.6, 19, 0, "-124"},
+		// 2**62
+		{4.611686018427387904e+18, 19, 0, "4611686018427387904"},
+		{-4.611686018427387904e+18, 19, 0, "-4611686018427387904"},
+		// 2**63
+		{9.223372036854775808e+18, 19, 0, "9223372036854775808"},
+		{-9.223372036854775808e+18, 19, 0, "-9223372036854775808"},
+		// 2**64
+		{1.8446744073709551616e+19, 20, 0, "18446744073709551616"},
+		{-1.8446744073709551616e+19, 20, 0, "-18446744073709551616"},
+	}
+
+	t.Run("float64", func(t *testing.T) {
+		for _, tt := range tests {
+			t.Run(tt.expected, func(t *testing.T) {
+				n, err := decimal128.FromFloat64(tt.val, tt.precision, tt.scale)
+				assert.NoError(t, err)
+
+				assert.Equal(t, tt.expected, big.NewFloat(n.ToFloat64(tt.scale)).Text('f', int(tt.scale)))
+			})
+		}
+
+		t.Run("large values", func(t *testing.T) {
+			// test entire float64 range
+			for scale := int32(-308); scale <= 308; scale++ {
+				val := math.Pow10(int(scale))
+				n, err := decimal128.FromFloat64(val, 1, -scale)
+				assert.NoError(t, err)
+				assert.Equal(t, "1", n.BigInt().String())
+			}
+
+			for scale := int32(-307); scale <= 306; scale++ {
+				val := 123 * math.Pow10(int(scale))
+				n, err := decimal128.FromFloat64(val, 2, -scale-1)
+				assert.NoError(t, err)
+				assert.Equal(t, "12", n.BigInt().String())
+				n, err = decimal128.FromFloat64(val, 3, -scale)
+				assert.NoError(t, err)
+				assert.Equal(t, "123", n.BigInt().String())
+				n, err = decimal128.FromFloat64(val, 4, -scale+1)
+				assert.NoError(t, err)
+				assert.Equal(t, "1230", n.BigInt().String())
+			}
+		})
+	})
+
+	t.Run("float32", func(t *testing.T) {
+		for _, tt := range tests {
+			t.Run(tt.expected, func(t *testing.T) {
+				n, err := decimal128.FromFloat32(float32(tt.val), tt.precision, tt.scale)
+				assert.NoError(t, err)
+
+				assert.Equal(t, tt.expected, big.NewFloat(float64(n.ToFloat32(tt.scale))).Text('f', int(tt.scale)))
+			})
+		}
+
+		t.Run("large values", func(t *testing.T) {
+			// test entire float32 range
+			for scale := int32(-38); scale <= 38; scale++ {
+				val := float32(math.Pow10(int(scale)))
+				n, err := decimal128.FromFloat32(val, 1, -scale)
+				assert.NoError(t, err)
+				assert.Equal(t, "1", n.BigInt().String())
+			}
+
+			for scale := int32(-37); scale <= 36; scale++ {
+				val := 123 * float32(math.Pow10(int(scale)))
+				n, err := decimal128.FromFloat32(val, 2, -scale-1)
+				assert.NoError(t, err)
+				assert.Equal(t, "12", n.BigInt().String())
+				n, err = decimal128.FromFloat32(val, 3, -scale)
+				assert.NoError(t, err)
+				assert.Equal(t, "123", n.BigInt().String())
+				n, err = decimal128.FromFloat32(val, 4, -scale+1)
+				assert.NoError(t, err)
+				assert.Equal(t, "1230", n.BigInt().String())
+			}
+		})
+	})
+}