You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by sb...@apache.org on 2019/06/14 12:36:02 UTC

[arrow] branch master updated: ARROW-4974: [Go] implement ArrayApproxEqual

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

sbinet 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 3cee652  ARROW-4974: [Go] implement ArrayApproxEqual
3cee652 is described below

commit 3cee652fa40718a4fb16e4ecf331daa0ca8c53d5
Author: Sebastien Binet <bi...@cern.ch>
AuthorDate: Fri Jun 14 14:35:42 2019 +0200

    ARROW-4974: [Go] implement ArrayApproxEqual
    
    Author: Sebastien Binet <bi...@cern.ch>
    
    Closes #4556 from sbinet/issue-4974 and squashes the following commits:
    
    f1baaf9dd <Sebastien Binet> ARROW-4974:  implement ArrayApproxEqual
---
 go/arrow/array/compare.go      | 256 +++++++++++++++++++++++++++++++++++++++++
 go/arrow/array/compare_test.go | 253 ++++++++++++++++++++++++++++++++++++++++
 2 files changed, 509 insertions(+)

diff --git a/go/arrow/array/compare.go b/go/arrow/array/compare.go
index 60e21fb..9fa13a1 100644
--- a/go/arrow/array/compare.go
+++ b/go/arrow/array/compare.go
@@ -17,7 +17,10 @@
 package array
 
 import (
+	"math"
+
 	"github.com/apache/arrow/go/arrow"
+	"github.com/apache/arrow/go/arrow/float16"
 	"github.com/pkg/errors"
 )
 
@@ -124,6 +127,175 @@ func ArraySliceEqual(left Interface, lbeg, lend int64, right Interface, rbeg, re
 	return ArrayEqual(l, r)
 }
 
+const defaultAbsoluteTolerance = 1e-5
+
+type equalOption struct {
+	atol   float64 // absolute tolerance
+	nansEq bool    // whether NaNs are considered equal.
+}
+
+func (eq equalOption) f16(f1, f2 float16.Num) bool {
+	v1 := float64(f1.Float32())
+	v2 := float64(f2.Float32())
+	switch {
+	case eq.nansEq:
+		return math.Abs(v1-v2) <= eq.atol || (math.IsNaN(v1) && math.IsNaN(v2))
+	default:
+		return math.Abs(v1-v2) <= eq.atol
+	}
+}
+
+func (eq equalOption) f32(f1, f2 float32) bool {
+	v1 := float64(f1)
+	v2 := float64(f2)
+	switch {
+	case eq.nansEq:
+		return math.Abs(v1-v2) <= eq.atol || (math.IsNaN(v1) && math.IsNaN(v2))
+	default:
+		return math.Abs(v1-v2) <= eq.atol
+	}
+}
+
+func (eq equalOption) f64(v1, v2 float64) bool {
+	switch {
+	case eq.nansEq:
+		return math.Abs(v1-v2) <= eq.atol || (math.IsNaN(v1) && math.IsNaN(v2))
+	default:
+		return math.Abs(v1-v2) <= eq.atol
+	}
+}
+
+func newEqualOption(opts ...EqualOption) equalOption {
+	eq := equalOption{
+		atol:   defaultAbsoluteTolerance,
+		nansEq: false,
+	}
+	for _, opt := range opts {
+		opt(&eq)
+	}
+
+	return eq
+}
+
+// EqualOption is a functional option type used to configure how Records and Arrays are compared.
+type EqualOption func(*equalOption)
+
+// WithNaNsEqual configures the comparison functions so that NaNs are considered equal.
+func WithNaNsEqual(v bool) EqualOption {
+	return func(o *equalOption) {
+		o.nansEq = v
+	}
+}
+
+// WithAbsTolerance configures the comparison functions so that 2 floating point values
+// v1 and v2 are considered equal if |v1-v2| <= atol.
+func WithAbsTolerance(atol float64) EqualOption {
+	return func(o *equalOption) {
+		o.atol = atol
+	}
+}
+
+// ArrayApproxEqual reports whether the two provided arrays are approximately equal.
+// For non-floating point arrays, it is equivalent to ArrayEqual.
+func ArrayApproxEqual(left, right Interface, opts ...EqualOption) bool {
+	opt := newEqualOption(opts...)
+	return arrayApproxEqual(left, right, opt)
+}
+
+func arrayApproxEqual(left, right Interface, opt equalOption) bool {
+	switch {
+	case !baseArrayEqual(left, right):
+		return false
+	case left.Len() == 0:
+		return true
+	case left.NullN() == left.Len():
+		return true
+	}
+
+	// at this point, we know both arrays have same type, same length, same number of nulls
+	// and nulls at the same place.
+	// compare the values.
+
+	switch l := left.(type) {
+	case *Null:
+		return true
+	case *Boolean:
+		r := right.(*Boolean)
+		return arrayEqualBoolean(l, r)
+	case *FixedSizeBinary:
+		r := right.(*FixedSizeBinary)
+		return arrayEqualFixedSizeBinary(l, r)
+	case *Binary:
+		r := right.(*Binary)
+		return arrayEqualBinary(l, r)
+	case *String:
+		r := right.(*String)
+		return arrayEqualString(l, r)
+	case *Int8:
+		r := right.(*Int8)
+		return arrayEqualInt8(l, r)
+	case *Int16:
+		r := right.(*Int16)
+		return arrayEqualInt16(l, r)
+	case *Int32:
+		r := right.(*Int32)
+		return arrayEqualInt32(l, r)
+	case *Int64:
+		r := right.(*Int64)
+		return arrayEqualInt64(l, r)
+	case *Uint8:
+		r := right.(*Uint8)
+		return arrayEqualUint8(l, r)
+	case *Uint16:
+		r := right.(*Uint16)
+		return arrayEqualUint16(l, r)
+	case *Uint32:
+		r := right.(*Uint32)
+		return arrayEqualUint32(l, r)
+	case *Uint64:
+		r := right.(*Uint64)
+		return arrayEqualUint64(l, r)
+	case *Float16:
+		r := right.(*Float16)
+		return arrayApproxEqualFloat16(l, r, opt)
+	case *Float32:
+		r := right.(*Float32)
+		return arrayApproxEqualFloat32(l, r, opt)
+	case *Float64:
+		r := right.(*Float64)
+		return arrayApproxEqualFloat64(l, r, opt)
+	case *Date32:
+		r := right.(*Date32)
+		return arrayEqualDate32(l, r)
+	case *Date64:
+		r := right.(*Date64)
+		return arrayEqualDate64(l, r)
+	case *Time32:
+		r := right.(*Time32)
+		return arrayEqualTime32(l, r)
+	case *Time64:
+		r := right.(*Time64)
+		return arrayEqualTime64(l, r)
+	case *Timestamp:
+		r := right.(*Timestamp)
+		return arrayEqualTimestamp(l, r)
+	case *List:
+		r := right.(*List)
+		return arrayApproxEqualList(l, r, opt)
+	case *FixedSizeList:
+		r := right.(*FixedSizeList)
+		return arrayApproxEqualFixedSizeList(l, r, opt)
+	case *Struct:
+		r := right.(*Struct)
+		return arrayApproxEqualStruct(l, r, opt)
+
+	default:
+		panic(errors.Errorf("arrow/array: unknown array type %T", l))
+	}
+
+	return false
+}
+
 func baseArrayEqual(left, right Interface) bool {
 	switch {
 	case left.Len() != right.Len():
@@ -151,3 +323,87 @@ func validityBitmapEqual(left, right Interface) bool {
 	}
 	return true
 }
+
+func arrayApproxEqualFloat16(left, right *Float16, opt equalOption) bool {
+	for i := 0; i < left.Len(); i++ {
+		if left.IsNull(i) {
+			continue
+		}
+		if !opt.f16(left.Value(i), right.Value(i)) {
+			return false
+		}
+	}
+	return true
+}
+
+func arrayApproxEqualFloat32(left, right *Float32, opt equalOption) bool {
+	for i := 0; i < left.Len(); i++ {
+		if left.IsNull(i) {
+			continue
+		}
+		if !opt.f32(left.Value(i), right.Value(i)) {
+			return false
+		}
+	}
+	return true
+}
+
+func arrayApproxEqualFloat64(left, right *Float64, opt equalOption) bool {
+	for i := 0; i < left.Len(); i++ {
+		if left.IsNull(i) {
+			continue
+		}
+		if !opt.f64(left.Value(i), right.Value(i)) {
+			return false
+		}
+	}
+	return true
+}
+
+func arrayApproxEqualList(left, right *List, opt equalOption) bool {
+	for i := 0; i < left.Len(); i++ {
+		if left.IsNull(i) {
+			continue
+		}
+		o := func() bool {
+			l := left.newListValue(i)
+			defer l.Release()
+			r := right.newListValue(i)
+			defer r.Release()
+			return arrayApproxEqual(l, r, opt)
+		}()
+		if !o {
+			return false
+		}
+	}
+	return true
+}
+
+func arrayApproxEqualFixedSizeList(left, right *FixedSizeList, opt equalOption) bool {
+	for i := 0; i < left.Len(); i++ {
+		if left.IsNull(i) {
+			continue
+		}
+		o := func() bool {
+			l := left.newListValue(i)
+			defer l.Release()
+			r := right.newListValue(i)
+			defer r.Release()
+			return arrayApproxEqual(l, r, opt)
+		}()
+		if !o {
+			return false
+		}
+	}
+	return true
+}
+
+func arrayApproxEqualStruct(left, right *Struct, opt equalOption) bool {
+	for i, lf := range left.fields {
+		rf := right.fields[i]
+		if !arrayApproxEqual(lf, rf, opt) {
+			return false
+		}
+	}
+	return true
+}
diff --git a/go/arrow/array/compare_test.go b/go/arrow/array/compare_test.go
index 33a5c92..9985f51 100644
--- a/go/arrow/array/compare_test.go
+++ b/go/arrow/array/compare_test.go
@@ -17,9 +17,12 @@
 package array_test
 
 import (
+	"fmt"
+	"math"
 	"testing"
 
 	"github.com/apache/arrow/go/arrow/array"
+	"github.com/apache/arrow/go/arrow/float16"
 	"github.com/apache/arrow/go/arrow/internal/arrdata"
 	"github.com/apache/arrow/go/arrow/memory"
 )
@@ -79,6 +82,256 @@ func TestArraySliceEqual(t *testing.T) {
 	}
 }
 
+func TestArrayApproxEqual(t *testing.T) {
+	for name, recs := range arrdata.Records {
+		t.Run(name, func(t *testing.T) {
+			rec := recs[0]
+			schema := rec.Schema()
+			for i, col := range rec.Columns() {
+				t.Run(schema.Field(i).Name, func(t *testing.T) {
+					arr := col
+					if !array.ArrayApproxEqual(arr, arr) {
+						t.Fatalf("identical arrays should compare equal:\narray=%v", arr)
+					}
+					sub1 := array.NewSlice(arr, 1, int64(arr.Len()))
+					defer sub1.Release()
+
+					sub2 := array.NewSlice(arr, 0, int64(arr.Len()-1))
+					defer sub2.Release()
+
+					if array.ArrayApproxEqual(sub1, sub2) {
+						t.Fatalf("non-identical arrays should not compare equal:\nsub1=%v\nsub2=%v\narrf=%v\n", sub1, sub2, arr)
+					}
+				})
+			}
+		})
+	}
+}
+
+func TestArrayApproxEqualFloats(t *testing.T) {
+	f16sFrom := func(vs []float64) []float16.Num {
+		o := make([]float16.Num, len(vs))
+		for i, v := range vs {
+			o[i] = float16.New(float32(v))
+		}
+		return o
+	}
+
+	for _, tc := range []struct {
+		name string
+		a1   interface{}
+		a2   interface{}
+		opts []array.EqualOption
+		want bool
+	}{
+		{
+			name: "f16",
+			a1:   f16sFrom([]float64{1, 2, 3, 4, 5, 6}),
+			a2:   f16sFrom([]float64{1, 2, 3, 4, 5, 6}),
+			want: true,
+		},
+		{
+			name: "f16-no-tol",
+			a1:   f16sFrom([]float64{1, 2, 3, 4, 5, 6}),
+			a2:   f16sFrom([]float64{1, 2, 3, 4, 5, 7}),
+			want: false,
+		},
+		{
+			name: "f16-tol-ok",
+			a1:   f16sFrom([]float64{1, 2, 3, 4, 5, 6}),
+			a2:   f16sFrom([]float64{1, 2, 3, 4, 5, 7}),
+			opts: []array.EqualOption{array.WithAbsTolerance(1)},
+			want: true,
+		},
+		{
+			name: "f16-nan",
+			a1:   f16sFrom([]float64{1, 2, 3, 4, 5, 6}),
+			a2:   f16sFrom([]float64{1, 2, 3, 4, 5, math.NaN()}),
+			want: false,
+		},
+		{
+			name: "f16-nan-not",
+			a1:   f16sFrom([]float64{1, 2, 3, 4, 5, 6}),
+			a2:   f16sFrom([]float64{1, 2, 3, 4, 5, math.NaN()}),
+			opts: []array.EqualOption{array.WithNaNsEqual(true)},
+			want: false,
+		},
+		{
+			name: "f16-nan-ok",
+			a1:   f16sFrom([]float64{1, 2, 3, 4, 5, math.NaN()}),
+			a2:   f16sFrom([]float64{1, 2, 3, 4, 5, math.NaN()}),
+			opts: []array.EqualOption{array.WithNaNsEqual(true)},
+			want: true,
+		},
+		{
+			name: "f16-nan-no-tol",
+			a1:   f16sFrom([]float64{1, 2, 3, 4, 5, math.NaN()}),
+			a2:   f16sFrom([]float64{1, 2, 3, 4, 6, math.NaN()}),
+			opts: []array.EqualOption{array.WithNaNsEqual(true)},
+			want: false,
+		},
+		{
+			name: "f16-nan-tol",
+			a1:   f16sFrom([]float64{1, 2, 3, 4, 5, math.NaN()}),
+			a2:   f16sFrom([]float64{1, 2, 3, 4, 6, math.NaN()}),
+			opts: []array.EqualOption{array.WithNaNsEqual(true), array.WithAbsTolerance(1)},
+			want: true,
+		},
+		{
+			name: "f32",
+			a1:   []float32{1, 2, 3, 4, 5, 6},
+			a2:   []float32{1, 2, 3, 4, 5, 6},
+			want: true,
+		},
+		{
+			name: "f32-no-tol",
+			a1:   []float32{1, 2, 3, 4, 5, 6},
+			a2:   []float32{1, 2, 3, 4, 5, 7},
+			want: false,
+		},
+		{
+			name: "f32-tol-ok",
+			a1:   []float32{1, 2, 3, 4, 5, 6},
+			a2:   []float32{1, 2, 3, 4, 5, 7},
+			opts: []array.EqualOption{array.WithAbsTolerance(1)},
+			want: true,
+		},
+		{
+			name: "f32-nan",
+			a1:   []float32{1, 2, 3, 4, 5, 6},
+			a2:   []float32{1, 2, 3, 4, 5, float32(math.NaN())},
+			want: false,
+		},
+		{
+			name: "f32-nan-not",
+			a1:   []float32{1, 2, 3, 4, 5, 6},
+			a2:   []float32{1, 2, 3, 4, 5, float32(math.NaN())},
+			opts: []array.EqualOption{array.WithNaNsEqual(true)},
+			want: false,
+		},
+		{
+			name: "f32-nan-ok",
+			a1:   []float32{1, 2, 3, 4, 5, float32(math.NaN())},
+			a2:   []float32{1, 2, 3, 4, 5, float32(math.NaN())},
+			opts: []array.EqualOption{array.WithNaNsEqual(true)},
+			want: true,
+		},
+		{
+			name: "f32-nan-no-tol",
+			a1:   []float32{1, 2, 3, 4, 5, float32(math.NaN())},
+			a2:   []float32{1, 2, 3, 4, 6, float32(math.NaN())},
+			opts: []array.EqualOption{array.WithNaNsEqual(true)},
+			want: false,
+		},
+		{
+			name: "f32-nan-tol",
+			a1:   []float32{1, 2, 3, 4, 5, float32(math.NaN())},
+			a2:   []float32{1, 2, 3, 4, 6, float32(math.NaN())},
+			opts: []array.EqualOption{array.WithNaNsEqual(true), array.WithAbsTolerance(1)},
+			want: true,
+		},
+		{
+			name: "f64",
+			a1:   []float64{1, 2, 3, 4, 5, 6},
+			a2:   []float64{1, 2, 3, 4, 5, 6},
+			want: true,
+		},
+		{
+			name: "f64-no-tol",
+			a1:   []float64{1, 2, 3, 4, 5, 6},
+			a2:   []float64{1, 2, 3, 4, 5, 7},
+			want: false,
+		},
+		{
+			name: "f64-tol-ok",
+			a1:   []float64{1, 2, 3, 4, 5, 6},
+			a2:   []float64{1, 2, 3, 4, 5, 7},
+			opts: []array.EqualOption{array.WithAbsTolerance(1)},
+			want: true,
+		},
+		{
+			name: "f64-nan",
+			a1:   []float64{1, 2, 3, 4, 5, 6},
+			a2:   []float64{1, 2, 3, 4, 5, math.NaN()},
+			want: false,
+		},
+		{
+			name: "f64-nan-not",
+			a1:   []float64{1, 2, 3, 4, 5, 6},
+			a2:   []float64{1, 2, 3, 4, 5, math.NaN()},
+			opts: []array.EqualOption{array.WithNaNsEqual(true)},
+			want: false,
+		},
+		{
+			name: "f64-nan-ok",
+			a1:   []float64{1, 2, 3, 4, 5, math.NaN()},
+			a2:   []float64{1, 2, 3, 4, 5, math.NaN()},
+			opts: []array.EqualOption{array.WithNaNsEqual(true)},
+			want: true,
+		},
+		{
+			name: "f64-nan-no-tol",
+			a1:   []float64{1, 2, 3, 4, 5, math.NaN()},
+			a2:   []float64{1, 2, 3, 4, 6, math.NaN()},
+			opts: []array.EqualOption{array.WithNaNsEqual(true)},
+			want: false,
+		},
+		{
+			name: "f64-nan-tol",
+			a1:   []float64{1, 2, 3, 4, 5, math.NaN()},
+			a2:   []float64{1, 2, 3, 4, 6, math.NaN()},
+			opts: []array.EqualOption{array.WithNaNsEqual(true), array.WithAbsTolerance(1)},
+			want: true,
+		},
+	} {
+		t.Run(tc.name, func(t *testing.T) {
+			mem := memory.NewCheckedAllocator(memory.NewGoAllocator())
+			defer mem.AssertSize(t, 0)
+
+			a1 := arrayOf(mem, tc.a1, nil)
+			defer a1.Release()
+			a2 := arrayOf(mem, tc.a2, nil)
+			defer a2.Release()
+
+			if got, want := array.ArrayApproxEqual(a1, a2, tc.opts...), tc.want; got != want {
+				t.Fatalf("invalid comparison: got=%v, want=%v\na1: %v\na2: %v\n", got, want, a1, a2)
+			}
+		})
+	}
+}
+
+func arrayOf(mem memory.Allocator, a interface{}, valids []bool) array.Interface {
+	if mem == nil {
+		mem = memory.NewGoAllocator()
+	}
+
+	switch a := a.(type) {
+	case []float16.Num:
+		bldr := array.NewFloat16Builder(mem)
+		defer bldr.Release()
+
+		bldr.AppendValues(a, valids)
+		return bldr.NewFloat16Array()
+
+	case []float32:
+		bldr := array.NewFloat32Builder(mem)
+		defer bldr.Release()
+
+		bldr.AppendValues(a, valids)
+		return bldr.NewFloat32Array()
+
+	case []float64:
+		bldr := array.NewFloat64Builder(mem)
+		defer bldr.Release()
+
+		bldr.AppendValues(a, valids)
+		return bldr.NewFloat64Array()
+
+	default:
+		panic(fmt.Errorf("arrdata: invalid data slice type %T", a))
+	}
+}
+
 func TestArrayEqualBaseArray(t *testing.T) {
 	mem := memory.NewCheckedAllocator(memory.NewGoAllocator())
 	defer mem.AssertSize(t, 0)