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


##########
go/arrow/compute/arithmetic.go:
##########
@@ -0,0 +1,155 @@
+// 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.
+
+package compute
+
+import (
+       "context"
+       "fmt"
+       "strings"
+
+       "github.com/apache/arrow/go/v10/arrow"
+       "github.com/apache/arrow/go/v10/arrow/compute/internal/exec"
+       "github.com/apache/arrow/go/v10/arrow/compute/internal/kernels"
+)
+
+type arithmeticFunction struct {
+       ScalarFunction
+}
+
+func (fn *arithmeticFunction) checkDecimals(vals ...arrow.DataType) error {
+       if !hasDecimal(vals...) {
+               return nil
+       }
+
+       if len(vals) != 2 {
+               return nil
+       }
+
+       op := fn.name[:strings.Index(fn.name, "_")]

Review Comment:
   Using string comparison for this is a bit weird, no? Can't you have 
`arithmeticFunction` be some kind of polymorphic type with different 
implementations of argument promotion for each subtype/implementation/whatever?



##########
go/arrow/compute/arithmetic.go:
##########
@@ -0,0 +1,155 @@
+// 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.
+
+package compute
+
+import (
+       "context"
+       "fmt"
+       "strings"
+
+       "github.com/apache/arrow/go/v10/arrow"
+       "github.com/apache/arrow/go/v10/arrow/compute/internal/exec"
+       "github.com/apache/arrow/go/v10/arrow/compute/internal/kernels"
+)
+
+type arithmeticFunction struct {
+       ScalarFunction
+}
+
+func (fn *arithmeticFunction) checkDecimals(vals ...arrow.DataType) error {
+       if !hasDecimal(vals...) {
+               return nil
+       }
+
+       if len(vals) != 2 {
+               return nil
+       }
+
+       op := fn.name[:strings.Index(fn.name, "_")]
+       switch op {
+       case "add", "subtract":
+               return castBinaryDecimalArgs(decPromoteAdd, vals...)
+       case "multiply":
+               return castBinaryDecimalArgs(decPromoteMultiply, vals...)
+       case "divide":
+               return castBinaryDecimalArgs(decPromoteDivide, vals...)
+       default:
+               return fmt.Errorf("%w: invalid decimal function: %s", 
arrow.ErrInvalid, fn.name)
+       }
+}
+
+func (fn *arithmeticFunction) DispatchBest(vals ...arrow.DataType) 
(exec.Kernel, error) {
+       if err := fn.checkArity(len(vals)); err != nil {
+               return nil, err
+       }
+
+       if err := fn.checkDecimals(vals...); err != nil {
+               return nil, err
+       }
+
+       if kn, err := fn.DispatchExact(vals...); err == nil {
+               return kn, nil
+       }
+
+       ensureDictionaryDecoded(vals...)
+
+       // only promote types for binary funcs
+       if len(vals) == 2 {
+               replaceNullWithOtherType(vals...)
+               if unit, istime := commonTemporalResolution(vals...); istime {
+                       replaceTemporalTypes(unit, vals...)
+               } else {
+                       if dt := commonNumeric(vals...); dt != nil {
+                               replaceTypes(dt, vals...)
+                       }
+               }
+       }
+
+       return fn.DispatchExact(vals...)
+}
+
+var (
+       addDoc FunctionDoc
+)
+
+func RegisterScalarArithmetic(reg FunctionRegistry) {
+       addFn := &arithmeticFunction{*NewScalarFunction("add", Binary(), 
addDoc)}
+       for _, k := range kernels.GetArithmeticKernels(kernels.OpAdd) {
+               if err := addFn.AddKernel(k); err != nil {
+                       panic(err)
+               }
+       }
+
+       reg.AddFunction(addFn, false)
+
+       addCheckedFn := &arithmeticFunction{*NewScalarFunction("add_checked", 
Binary(), addDoc)}

Review Comment:
   I don't know how much you want to follow the C++ design but one mistake we 
made is that the "default" arithmetic functions are unchecked.
   If I were to do it again I'd have "add" (checked) and "add_unchecked".



##########
go/arrow/compute/arithmetic_test.go:
##########
@@ -0,0 +1,229 @@
+// 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.
+
+package compute_test
+
+import (
+       "context"
+       "fmt"
+       "strings"
+       "testing"
+
+       "github.com/apache/arrow/go/v10/arrow"
+       "github.com/apache/arrow/go/v10/arrow/array"
+       "github.com/apache/arrow/go/v10/arrow/compute"
+       "github.com/apache/arrow/go/v10/arrow/compute/internal/exec"
+       "github.com/apache/arrow/go/v10/arrow/memory"
+       "github.com/apache/arrow/go/v10/arrow/scalar"
+       "github.com/stretchr/testify/suite"
+)
+
+type binaryFunc = func(context.Context, compute.ArithmeticOptions, 
compute.Datum, compute.Datum) (compute.Datum, error)
+
+type BinaryArithmeticSuite[T exec.NumericTypes] struct {
+       suite.Suite
+
+       mem  *memory.CheckedAllocator
+       opts compute.ArithmeticOptions
+       ctx  context.Context
+}
+
+func (BinaryArithmeticSuite[T]) DataType() arrow.DataType {
+       return exec.GetDataType[T]()
+}
+
+func (b *BinaryArithmeticSuite[T]) SetupTest() {
+       b.mem = memory.NewCheckedAllocator(memory.DefaultAllocator)
+       b.opts.CheckOverflow = false
+       b.ctx = compute.WithAllocator(context.TODO(), b.mem)
+}
+
+func (b *BinaryArithmeticSuite[T]) TearDownTest() {
+       b.mem.AssertSize(b.T(), 0)
+}
+
+func (b *BinaryArithmeticSuite[T]) makeNullScalar() scalar.Scalar {
+       return scalar.MakeNullScalar(b.DataType())
+}
+
+func (b *BinaryArithmeticSuite[T]) makeScalar(val T) scalar.Scalar {
+       return scalar.MakeScalar(val)
+}
+
+func (b *BinaryArithmeticSuite[T]) assertBinopScalars(fn binaryFunc, lhs, rhs 
T, expected T) {
+       left, right := b.makeScalar(lhs), b.makeScalar(rhs)
+       exp := b.makeScalar(expected)
+
+       actual, err := fn(b.ctx, b.opts, &compute.ScalarDatum{Value: left}, 
&compute.ScalarDatum{Value: right})
+       b.NoError(err)
+       sc := actual.(*compute.ScalarDatum).Value
+
+       b.Truef(scalar.Equals(exp, sc), "expected: %s\ngot: %s", exp, sc)
+}
+
+func (b *BinaryArithmeticSuite[T]) assertBinopScArr(fn binaryFunc, lhs T, rhs, 
expected string) {

Review Comment:
   "Sc" vs "Scalar" is not really meaningful to the reader.



##########
go/arrow/compute/arithmetic_test.go:
##########
@@ -0,0 +1,229 @@
+// 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.
+
+package compute_test
+
+import (
+       "context"
+       "fmt"
+       "strings"
+       "testing"
+
+       "github.com/apache/arrow/go/v10/arrow"
+       "github.com/apache/arrow/go/v10/arrow/array"
+       "github.com/apache/arrow/go/v10/arrow/compute"
+       "github.com/apache/arrow/go/v10/arrow/compute/internal/exec"
+       "github.com/apache/arrow/go/v10/arrow/memory"
+       "github.com/apache/arrow/go/v10/arrow/scalar"
+       "github.com/stretchr/testify/suite"
+)
+
+type binaryFunc = func(context.Context, compute.ArithmeticOptions, 
compute.Datum, compute.Datum) (compute.Datum, error)
+
+type BinaryArithmeticSuite[T exec.NumericTypes] struct {
+       suite.Suite
+
+       mem  *memory.CheckedAllocator
+       opts compute.ArithmeticOptions
+       ctx  context.Context
+}
+
+func (BinaryArithmeticSuite[T]) DataType() arrow.DataType {
+       return exec.GetDataType[T]()
+}
+
+func (b *BinaryArithmeticSuite[T]) SetupTest() {
+       b.mem = memory.NewCheckedAllocator(memory.DefaultAllocator)
+       b.opts.CheckOverflow = false
+       b.ctx = compute.WithAllocator(context.TODO(), b.mem)
+}
+
+func (b *BinaryArithmeticSuite[T]) TearDownTest() {
+       b.mem.AssertSize(b.T(), 0)
+}
+
+func (b *BinaryArithmeticSuite[T]) makeNullScalar() scalar.Scalar {
+       return scalar.MakeNullScalar(b.DataType())
+}
+
+func (b *BinaryArithmeticSuite[T]) makeScalar(val T) scalar.Scalar {
+       return scalar.MakeScalar(val)
+}
+
+func (b *BinaryArithmeticSuite[T]) assertBinopScalars(fn binaryFunc, lhs, rhs 
T, expected T) {
+       left, right := b.makeScalar(lhs), b.makeScalar(rhs)
+       exp := b.makeScalar(expected)
+
+       actual, err := fn(b.ctx, b.opts, &compute.ScalarDatum{Value: left}, 
&compute.ScalarDatum{Value: right})
+       b.NoError(err)
+       sc := actual.(*compute.ScalarDatum).Value
+
+       b.Truef(scalar.Equals(exp, sc), "expected: %s\ngot: %s", exp, sc)

Review Comment:
   I think you want to define helper testers/assertions for this ASAP, 
otherwise you'll write such assertions a lot in your test code.



##########
go/arrow/compute/arithmetic_test.go:
##########
@@ -0,0 +1,229 @@
+// 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.
+
+package compute_test
+
+import (
+       "context"
+       "fmt"
+       "strings"
+       "testing"
+
+       "github.com/apache/arrow/go/v10/arrow"
+       "github.com/apache/arrow/go/v10/arrow/array"
+       "github.com/apache/arrow/go/v10/arrow/compute"
+       "github.com/apache/arrow/go/v10/arrow/compute/internal/exec"
+       "github.com/apache/arrow/go/v10/arrow/memory"
+       "github.com/apache/arrow/go/v10/arrow/scalar"
+       "github.com/stretchr/testify/suite"
+)
+
+type binaryFunc = func(context.Context, compute.ArithmeticOptions, 
compute.Datum, compute.Datum) (compute.Datum, error)
+
+type BinaryArithmeticSuite[T exec.NumericTypes] struct {
+       suite.Suite
+
+       mem  *memory.CheckedAllocator
+       opts compute.ArithmeticOptions
+       ctx  context.Context
+}
+
+func (BinaryArithmeticSuite[T]) DataType() arrow.DataType {
+       return exec.GetDataType[T]()
+}
+
+func (b *BinaryArithmeticSuite[T]) SetupTest() {
+       b.mem = memory.NewCheckedAllocator(memory.DefaultAllocator)
+       b.opts.CheckOverflow = false
+       b.ctx = compute.WithAllocator(context.TODO(), b.mem)
+}
+
+func (b *BinaryArithmeticSuite[T]) TearDownTest() {
+       b.mem.AssertSize(b.T(), 0)
+}
+
+func (b *BinaryArithmeticSuite[T]) makeNullScalar() scalar.Scalar {
+       return scalar.MakeNullScalar(b.DataType())
+}
+
+func (b *BinaryArithmeticSuite[T]) makeScalar(val T) scalar.Scalar {
+       return scalar.MakeScalar(val)
+}
+
+func (b *BinaryArithmeticSuite[T]) assertBinopScalars(fn binaryFunc, lhs, rhs 
T, expected T) {
+       left, right := b.makeScalar(lhs), b.makeScalar(rhs)
+       exp := b.makeScalar(expected)
+
+       actual, err := fn(b.ctx, b.opts, &compute.ScalarDatum{Value: left}, 
&compute.ScalarDatum{Value: right})
+       b.NoError(err)
+       sc := actual.(*compute.ScalarDatum).Value
+
+       b.Truef(scalar.Equals(exp, sc), "expected: %s\ngot: %s", exp, sc)
+}
+
+func (b *BinaryArithmeticSuite[T]) assertBinopScArr(fn binaryFunc, lhs T, rhs, 
expected string) {
+       left := b.makeScalar(lhs)
+       b.assertBinopScalarArr(fn, left, rhs, expected)
+}
+
+func (b *BinaryArithmeticSuite[T]) assertBinopScalarArr(fn binaryFunc, lhs 
scalar.Scalar, rhs, expected string) {
+       right, _, _ := array.FromJSON(b.mem, b.DataType(), 
strings.NewReader(rhs))
+       defer right.Release()
+       exp, _, _ := array.FromJSON(b.mem, b.DataType(), 
strings.NewReader(expected))
+       defer exp.Release()
+
+       actual, err := fn(b.ctx, b.opts, &compute.ScalarDatum{Value: lhs}, 
&compute.ArrayDatum{Value: right.Data()})
+       b.NoError(err)
+       defer actual.Release()
+       assertDatumsEqual(b.T(), &compute.ArrayDatum{Value: exp.Data()}, actual)
+}
+
+func (b *BinaryArithmeticSuite[T]) assertBinopArrSc(fn binaryFunc, lhs string, 
rhs T, expected string) {

Review Comment:
   Same here.



##########
go/arrow/compute/arithmetic_test.go:
##########
@@ -0,0 +1,229 @@
+// 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.
+
+package compute_test
+
+import (
+       "context"
+       "fmt"
+       "strings"
+       "testing"
+
+       "github.com/apache/arrow/go/v10/arrow"
+       "github.com/apache/arrow/go/v10/arrow/array"
+       "github.com/apache/arrow/go/v10/arrow/compute"
+       "github.com/apache/arrow/go/v10/arrow/compute/internal/exec"
+       "github.com/apache/arrow/go/v10/arrow/memory"
+       "github.com/apache/arrow/go/v10/arrow/scalar"
+       "github.com/stretchr/testify/suite"
+)
+
+type binaryFunc = func(context.Context, compute.ArithmeticOptions, 
compute.Datum, compute.Datum) (compute.Datum, error)
+
+type BinaryArithmeticSuite[T exec.NumericTypes] struct {
+       suite.Suite
+
+       mem  *memory.CheckedAllocator
+       opts compute.ArithmeticOptions
+       ctx  context.Context
+}
+
+func (BinaryArithmeticSuite[T]) DataType() arrow.DataType {
+       return exec.GetDataType[T]()
+}
+
+func (b *BinaryArithmeticSuite[T]) SetupTest() {
+       b.mem = memory.NewCheckedAllocator(memory.DefaultAllocator)
+       b.opts.CheckOverflow = false
+       b.ctx = compute.WithAllocator(context.TODO(), b.mem)
+}
+
+func (b *BinaryArithmeticSuite[T]) TearDownTest() {
+       b.mem.AssertSize(b.T(), 0)
+}
+
+func (b *BinaryArithmeticSuite[T]) makeNullScalar() scalar.Scalar {
+       return scalar.MakeNullScalar(b.DataType())
+}
+
+func (b *BinaryArithmeticSuite[T]) makeScalar(val T) scalar.Scalar {
+       return scalar.MakeScalar(val)
+}
+
+func (b *BinaryArithmeticSuite[T]) assertBinopScalars(fn binaryFunc, lhs, rhs 
T, expected T) {
+       left, right := b.makeScalar(lhs), b.makeScalar(rhs)
+       exp := b.makeScalar(expected)
+
+       actual, err := fn(b.ctx, b.opts, &compute.ScalarDatum{Value: left}, 
&compute.ScalarDatum{Value: right})
+       b.NoError(err)
+       sc := actual.(*compute.ScalarDatum).Value
+
+       b.Truef(scalar.Equals(exp, sc), "expected: %s\ngot: %s", exp, sc)
+}
+
+func (b *BinaryArithmeticSuite[T]) assertBinopScArr(fn binaryFunc, lhs T, rhs, 
expected string) {
+       left := b.makeScalar(lhs)
+       b.assertBinopScalarArr(fn, left, rhs, expected)
+}
+
+func (b *BinaryArithmeticSuite[T]) assertBinopScalarArr(fn binaryFunc, lhs 
scalar.Scalar, rhs, expected string) {
+       right, _, _ := array.FromJSON(b.mem, b.DataType(), 
strings.NewReader(rhs))
+       defer right.Release()
+       exp, _, _ := array.FromJSON(b.mem, b.DataType(), 
strings.NewReader(expected))
+       defer exp.Release()
+
+       actual, err := fn(b.ctx, b.opts, &compute.ScalarDatum{Value: lhs}, 
&compute.ArrayDatum{Value: right.Data()})
+       b.NoError(err)
+       defer actual.Release()
+       assertDatumsEqual(b.T(), &compute.ArrayDatum{Value: exp.Data()}, actual)
+}
+
+func (b *BinaryArithmeticSuite[T]) assertBinopArrSc(fn binaryFunc, lhs string, 
rhs T, expected string) {
+       right := b.makeScalar(rhs)
+       b.assertBinopArrScalar(fn, lhs, right, expected)
+}
+
+func (b *BinaryArithmeticSuite[T]) assertBinopArrScalar(fn binaryFunc, lhs 
string, rhs scalar.Scalar, expected string) {
+       left, _, _ := array.FromJSON(b.mem, b.DataType(), 
strings.NewReader(lhs))
+       defer left.Release()
+       exp, _, _ := array.FromJSON(b.mem, b.DataType(), 
strings.NewReader(expected))
+       defer exp.Release()
+
+       actual, err := fn(b.ctx, b.opts, &compute.ArrayDatum{Value: 
left.Data()}, &compute.ScalarDatum{Value: rhs})
+       b.NoError(err)
+       defer actual.Release()
+       assertDatumsEqual(b.T(), &compute.ArrayDatum{Value: exp.Data()}, actual)
+}
+
+func (b *BinaryArithmeticSuite[T]) assertBinopArrays(fn binaryFunc, lhs, rhs, 
expected string) {
+       left, _, _ := array.FromJSON(b.mem, b.DataType(), 
strings.NewReader(lhs))
+       defer left.Release()
+       right, _, _ := array.FromJSON(b.mem, b.DataType(), 
strings.NewReader(rhs))
+       defer right.Release()
+       exp, _, _ := array.FromJSON(b.mem, b.DataType(), 
strings.NewReader(expected))
+       defer exp.Release()
+
+       b.assertBinop(fn, left, right, exp)
+}
+
+func (b *BinaryArithmeticSuite[T]) assertBinop(fn binaryFunc, left, right, 
expected arrow.Array) {
+       actual, err := fn(b.ctx, b.opts, &compute.ArrayDatum{Value: 
left.Data()}, &compute.ArrayDatum{Value: right.Data()})
+       b.Require().NoError(err)
+       defer actual.Release()
+       assertDatumsEqual(b.T(), &compute.ArrayDatum{Value: expected.Data()}, 
actual)
+
+       // also check (Scalar, Scalar) operations
+       for i := 0; i < expected.Len(); i++ {
+               s, err := scalar.GetScalar(expected, i)
+               b.Require().NoError(err)
+               lhs, _ := scalar.GetScalar(left, i)
+               rhs, _ := scalar.GetScalar(right, i)
+
+               actual, err := fn(b.ctx, b.opts, &compute.ScalarDatum{Value: 
lhs}, &compute.ScalarDatum{Value: rhs})
+               b.NoError(err)
+               b.Truef(scalar.Equals(s, actual.(*compute.ScalarDatum).Value), 
"expected: %s\ngot: %s", s, actual)
+       }
+}
+
+func (b *BinaryArithmeticSuite[T]) setOverflowCheck(value bool) {
+       b.opts.CheckOverflow = value
+}
+
+func (b *BinaryArithmeticSuite[T]) assertBinopErr(fn binaryFunc, lhs, rhs, 
expectedMsg string) {
+       left, _, _ := array.FromJSON(b.mem, b.DataType(), 
strings.NewReader(lhs))
+       defer left.Release()
+       right, _, _ := array.FromJSON(b.mem, b.DataType(), 
strings.NewReader(rhs))
+       defer right.Release()
+
+       _, err := fn(b.ctx, b.opts, &compute.ArrayDatum{left.Data()}, 
&compute.ArrayDatum{Value: right.Data()})
+       b.ErrorIs(err, arrow.ErrInvalid)
+       b.ErrorContains(err, expectedMsg)
+}
+
+func (b *BinaryArithmeticSuite[T]) TestAdd() {
+       b.Run(b.DataType().String(), func() {
+               for _, overflow := range []bool{false, true} {
+                       b.Run(fmt.Sprintf("overflow=%t", overflow), func() {
+                               b.setOverflowCheck(overflow)
+
+                               b.assertBinopArrays(compute.Add, `[]`, `[]`, 
`[]`)
+                               b.assertBinopArrays(compute.Add, `[3, 2, 6]`, 
`[1, 0, 2]`, `[4, 2, 8]`)
+                               // nulls on one side
+                               b.assertBinopArrays(compute.Add, `[null, 1, 
null]`, `[3, 4, 5]`, `[null, 5, null]`)
+                               b.assertBinopArrays(compute.Add, `[3, 4, 5]`, 
`[null, 1, null]`, `[null, 5, null]`)
+                               // nulls on both sides
+                               b.assertBinopArrays(compute.Add, `[null, 1, 
2]`, `[3, 4, null]`, `[null, 5, null]`)
+                               // all nulls
+                               b.assertBinopArrays(compute.Add, `[null]`, 
`[null]`, `[null]`)
+
+                               // scalar on the left
+                               b.assertBinopScArr(compute.Add, 3, `[1, 2]`, 
`[4, 5]`)
+                               b.assertBinopScArr(compute.Add, 3, `[null, 2]`, 
`[null, 5]`)
+                               b.assertBinopScalarArr(compute.Add, 
b.makeNullScalar(), `[1, 2]`, `[null, null]`)
+                               b.assertBinopScalarArr(compute.Add, 
b.makeNullScalar(), `[null, 2]`, `[null, null]`)
+                               // scalar on the right
+                               b.assertBinopArrSc(compute.Add, `[1, 2]`, 3, 
`[4, 5]`)
+                               b.assertBinopArrSc(compute.Add, `[null, 2]`, 3, 
`[null, 5]`)
+                               b.assertBinopArrScalar(compute.Add, `[1, 2]`, 
b.makeNullScalar(), `[null, null]`)
+                               b.assertBinopArrScalar(compute.Add, `[null, 
2]`, b.makeNullScalar(), `[null, null]`)
+                       })
+               }
+       })
+}
+
+func (b *BinaryArithmeticSuite[T]) TestSub() {
+       b.Run(b.DataType().String(), func() {
+               for _, overflow := range []bool{false, true} {
+                       b.Run(fmt.Sprintf("overflow=%t", overflow), func() {
+                               b.setOverflowCheck(overflow)
+
+                               b.assertBinopArrays(compute.Subtract, `[]`, 
`[]`, `[]`)
+                               b.assertBinopArrays(compute.Subtract, `[3, 2, 
6]`, `[1, 0, 2]`, `[2, 2, 4]`)
+                               // nulls on one side
+                               b.assertBinopArrays(compute.Subtract, `[null, 
4, null]`, `[2, 1, 0]`, `[null, 3, null]`)
+                               b.assertBinopArrays(compute.Subtract, `[3, 4, 
5]`, `[null, 1, null]`, `[null, 3, null]`)
+                               // nulls on both sides
+                               b.assertBinopArrays(compute.Subtract, `[null, 
4, 3]`, `[2, 1, null]`, `[null, 3, null]`)
+                               // all nulls
+                               b.assertBinopArrays(compute.Subtract, `[null]`, 
`[null]`, `[null]`)
+
+                               // scalar on the left
+                               b.assertBinopScArr(compute.Subtract, 3, `[1, 
2]`, `[2, 1]`)
+                               b.assertBinopScArr(compute.Subtract, 3, `[null, 
2]`, `[null, 1]`)
+                               b.assertBinopScalarArr(compute.Subtract, 
b.makeNullScalar(), `[1, 2]`, `[null, null]`)
+                               b.assertBinopScalarArr(compute.Subtract, 
b.makeNullScalar(), `[null, 2]`, `[null, null]`)
+                               // scalar on the right
+                               b.assertBinopArrSc(compute.Subtract, `[4, 5]`, 
3, `[1, 2]`)
+                               b.assertBinopArrSc(compute.Subtract, `[null, 
5]`, 3, `[null, 2]`)
+                               b.assertBinopArrScalar(compute.Subtract, `[1, 
2]`, b.makeNullScalar(), `[null, null]`)
+                               b.assertBinopArrScalar(compute.Subtract, 
`[null, 2]`, b.makeNullScalar(), `[null, null]`)
+                       })
+               }
+       })
+}
+
+func TestBinaryArithmetic(t *testing.T) {
+       suite.Run(t, new(BinaryArithmeticSuite[int8]))

Review Comment:
   Is there a way of looping other those types?
   
   Also, you need not necessarily use generics if you can simply define the 
datatype at runtime (which you can certainly do!). This can cut down on compile 
times in addition to allow easier looping on datatypes of choice (including 
parametric types such as temporals).



##########
go/arrow/compute/internal/exec/utils.go:
##########
@@ -172,6 +179,11 @@ func GetDataType[T NumericTypes | bool | string]() 
arrow.DataType {
        return typMap[reflect.TypeOf(z)]
 }
 
+func GetType[T NumericTypes | bool | string]() arrow.Type {
+       var z T
+       return typMap[reflect.TypeOf(z)].ID()

Review Comment:
   Does this happen purely at compile time or is there an actual map lookup at 
runtime?



##########
go/arrow/compute/internal/kernels/_lib/types.h:
##########
@@ -0,0 +1,477 @@
+// 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.
+
+// corresponds to datatype.go's arrow.Type
+enum class arrtype : int {
+    NULL,
+    BOOL,
+    UINT8,
+    INT8,
+    UINT16,
+    INT16,
+    UINT32,
+    INT32,
+    UINT64,
+    INT64,
+    FLOAT16,
+    FLOAT32,
+    FLOAT64
+};
+
+// The following is copied from <type_traits> since we use -target 

Review Comment:
   Copied from where exactly? What is the license?



##########
go/arrow/compute/internal/kernels/base_arithmetic.go:
##########
@@ -0,0 +1,141 @@
+// 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.
+
+package kernels
+
+import (
+       "fmt"
+       "math/bits"
+
+       "github.com/apache/arrow/go/v10/arrow"
+       "github.com/apache/arrow/go/v10/arrow/compute/internal/exec"
+       "github.com/apache/arrow/go/v10/arrow/internal/debug"
+       "golang.org/x/exp/constraints"
+)
+
+type ArithmeticOp int8
+
+const (
+       OpAdd ArithmeticOp = iota
+       OpAddChecked
+       OpSub
+       OpSubChecked
+)
+
+func getGoArithmeticBinaryOpsFloating[T constraints.Float](op ArithmeticOp) 
binaryOps[T, T, T] {

Review Comment:
   Why is this needed in addition to the C++ versions?



##########
go/arrow/compute/internal/kernels/_lib/base_arithmetic.cc:
##########
@@ -0,0 +1,255 @@
+// 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.
+
+#include <arch.h>
+#include <stdint.h>
+#include "types.h"
+#include "vendored/safe-math.h"
+
+// Define functions AddWithOverflow, SubtractWithOverflow, MultiplyWithOverflow
+// with the signature `bool(T u, T v, T* out)` where T is an integer type.
+// On overflow, these functions return true.  Otherwise, false is returned
+// and `out` is updated with the result of the operation.
+
+#define OP_WITH_OVERFLOW(_func_name, _psnip_op, _type, _psnip_type) \
+  static inline bool _func_name(_type u, _type v, _type* out) {     \
+    return !psnip_safe_##_psnip_type##_##_psnip_op(out, u, v);      \
+  }
+
+#define OPS_WITH_OVERFLOW(_func_name, _psnip_op)            \
+  OP_WITH_OVERFLOW(_func_name, _psnip_op, int8_t, int8)     \
+  OP_WITH_OVERFLOW(_func_name, _psnip_op, int16_t, int16)   \
+  OP_WITH_OVERFLOW(_func_name, _psnip_op, int32_t, int32)   \
+  OP_WITH_OVERFLOW(_func_name, _psnip_op, int64_t, int64)   \
+  OP_WITH_OVERFLOW(_func_name, _psnip_op, uint8_t, uint8)   \
+  OP_WITH_OVERFLOW(_func_name, _psnip_op, uint16_t, uint16) \
+  OP_WITH_OVERFLOW(_func_name, _psnip_op, uint32_t, uint32) \
+  OP_WITH_OVERFLOW(_func_name, _psnip_op, uint64_t, uint64)
+
+OPS_WITH_OVERFLOW(AddWithOverflow, add)
+OPS_WITH_OVERFLOW(SubtractWithOverflow, sub)
+OPS_WITH_OVERFLOW(MultiplyWithOverflow, mul)
+OPS_WITH_OVERFLOW(DivideWithOverflow, div)
+
+// Corresponds to equivalent ArithmeticOp enum in base_arithmetic.go
+// for passing across which operation to perform. This allows simpler
+// implementation at the cost of having to pass the extra int8 and
+// perform a switch.
+//
+// In cases of small arrays, this is completely negligible. In cases
+// of large arrays, the time saved by using SIMD here is significantly
+// worth the cost.
+enum class optype : int8_t {
+    ADD,
+    ADD_CHECKED,
+    SUB, 
+    SUB_CHECKED,
+};
+
+template <typename T>
+using is_unsigned_integer_value = bool_constant<is_integral_v<T> && 
is_unsigned_v<T>>;
+
+template <typename T>
+using is_signed_integer_value = bool_constant<is_integral_v<T> && 
is_signed_v<T>>;
+
+template <typename T, typename R = T>
+using enable_if_signed_integer_t = 
enable_if_t<is_signed_integer_value<T>::value, R>;
+
+template <typename T, typename R = T>
+using enable_if_unsigned_integer_t = 
enable_if_t<is_unsigned_integer_value<T>::value, R>;
+
+template <typename T, typename R = T>
+using enable_if_integer_t = enable_if_t<
+    is_signed_integer_value<T>::value || is_unsigned_integer_value<T>::value, 
R>;
+
+template <typename T, typename R = T>
+using enable_if_floating_t = enable_if_t<is_floating_point_v<T>, R>;
+
+struct Add {

Review Comment:
   You copied this directly from the Arrow C++ source code?
   Because now you can use C++17 features which will allow writing neater code, 
as @cyb70289 alluded to below.
   



##########
go/arrow/compute/internal/kernels/_lib/base_arithmetic_avx2_amd64.s:
##########
@@ -0,0 +1,12469 @@
+       .text

Review Comment:
   Can you mark generated files generated?
   
https://docs.github.com/en/repositories/working-with-files/managing-files/customizing-how-changed-files-appear-on-github



##########
go/arrow/compute/internal/kernels/base_arithmetic.go:
##########
@@ -0,0 +1,141 @@
+// 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.
+
+package kernels
+
+import (
+       "fmt"
+       "math/bits"
+
+       "github.com/apache/arrow/go/v10/arrow"
+       "github.com/apache/arrow/go/v10/arrow/compute/internal/exec"
+       "github.com/apache/arrow/go/v10/arrow/internal/debug"
+       "golang.org/x/exp/constraints"
+)
+
+type ArithmeticOp int8
+
+const (
+       OpAdd ArithmeticOp = iota
+       OpAddChecked
+       OpSub
+       OpSubChecked
+)
+
+func getGoArithmeticBinaryOpsFloating[T constraints.Float](op ArithmeticOp) 
binaryOps[T, T, T] {
+       Op := map[ArithmeticOp]func(a, b T, e *error) T{
+               OpAdd:        func(a, b T, _ *error) T { return a + b },
+               OpAddChecked: func(a, b T, _ *error) T { return a + b },
+               OpSub:        func(a, b T, _ *error) T { return a - b },
+               OpSubChecked: func(a, b T, _ *error) T { return a - b },
+       }[op]
+
+       return binaryOps[T, T, T]{
+               arrArr: func(_ *exec.KernelCtx, left, right, out []T) error {
+                       var err error
+                       for i := range out {
+                               out[i] = Op(left[i], right[i], &err)
+                       }
+                       return err
+               },
+               arrScalar: func(ctx *exec.KernelCtx, left []T, right T, out 
[]T) error {
+                       var err error
+                       for i := range out {
+                               out[i] = Op(left[i], right, &err)
+                       }
+                       return err
+               },
+               scalarArr: func(ctx *exec.KernelCtx, left T, right, out []T) 
error {
+                       var err error
+                       for i := range out {
+                               out[i] = Op(left, right[i], &err)
+                       }
+                       return err
+               },
+       }
+}
+
+func getGoArithmeticBinaryOpsIntegral[T exec.UintTypes | exec.IntTypes](op 
ArithmeticOp) binaryOps[T, T, T] {
+       Op := map[ArithmeticOp]func(a, b T, e *error) T{
+               OpAdd: func(a, b T, _ *error) T { return a + b },
+               OpAddChecked: func(a, b T, e *error) T {
+                       out, carry := bits.Add64(uint64(a), uint64(b), 0)
+                       if carry > 0 {
+                               *e = fmt.Errorf("%w: overflow", 
arrow.ErrInvalid)
+                       }
+                       return T(out)

Review Comment:
   Yeah... what if conversion truncates the result?



##########
go/arrow/compute/internal/kernels/helpers.go:
##########
@@ -159,6 +161,72 @@ func ScalarUnaryBoolArg[OutT exec.FixedWidthTypes](op 
func(*exec.KernelCtx, []by
        }
 }
 
+func UnboxScalar[T exec.FixedWidthTypes](val scalar.PrimitiveScalar) T {
+       return exec.GetData[T](val.Data())[0]
+}
+
+func UnboxBinaryScalar(val scalar.BinaryScalar) []byte {
+       if !val.IsValid() {
+               return nil
+       }
+       return val.Data()
+}
+
+type binaryOps[OutT, Arg0T, Arg1T exec.FixedWidthTypes] struct {
+       arrArr    func(*exec.KernelCtx, []Arg0T, []Arg1T, []OutT) error
+       arrScalar func(*exec.KernelCtx, []Arg0T, Arg1T, []OutT) error
+       scalarArr func(*exec.KernelCtx, Arg0T, []Arg1T, []OutT) error
+}
+
+func ScalarBinary[OutT, Arg0T, Arg1T exec.FixedWidthTypes](ops binaryOps[OutT, 
Arg0T, Arg1T]) exec.ArrayKernelExec {
+       arrayArray := func(ctx *exec.KernelCtx, arg0, arg1 *exec.ArraySpan, out 
*exec.ExecResult) error {
+               var (
+                       a0      = exec.GetSpanValues[Arg0T](arg0, 1)
+                       a1      = exec.GetSpanValues[Arg1T](arg1, 1)
+                       outData = exec.GetSpanValues[OutT](out, 1)
+               )
+               return ops.arrArr(ctx, a0, a1, outData)
+       }
+
+       arrayScalar := func(ctx *exec.KernelCtx, arg0 *exec.ArraySpan, arg1 
scalar.Scalar, out *exec.ExecResult) error {
+               var (
+                       a0      = exec.GetSpanValues[Arg0T](arg0, 1)
+                       a1      = 
UnboxScalar[Arg1T](arg1.(scalar.PrimitiveScalar))
+                       outData = exec.GetSpanValues[OutT](out, 1)
+               )
+               return ops.arrScalar(ctx, a0, a1, outData)
+       }
+
+       scalarArray := func(ctx *exec.KernelCtx, arg0 scalar.Scalar, arg1 
*exec.ArraySpan, out *exec.ExecResult) error {
+               var (
+                       a0      = 
UnboxScalar[Arg0T](arg0.(scalar.PrimitiveScalar))
+                       a1      = exec.GetSpanValues[Arg1T](arg1, 1)
+                       outData = exec.GetSpanValues[OutT](out, 1)
+               )
+               return ops.scalarArr(ctx, a0, a1, outData)
+       }
+
+       return func(ctx *exec.KernelCtx, batch *exec.ExecSpan, out 
*exec.ExecResult) error {
+               if batch.Values[0].IsArray() {
+                       if batch.Values[1].IsArray() {
+                               return arrayArray(ctx, &batch.Values[0].Array, 
&batch.Values[1].Array, out)
+                       }
+                       return arrayScalar(ctx, &batch.Values[0].Array, 
batch.Values[1].Scalar, out)
+               }
+
+               if batch.Values[1].IsArray() {
+                       return scalarArray(ctx, batch.Values[0].Scalar, 
&batch.Values[1].Array, out)
+               }
+
+               debug.Assert(false, "should be unreachable")
+               return fmt.Errorf("%w: scalar binary with two scalars?", 
arrow.ErrInvalid)
+       }
+}
+
+func ScalarBinaryEqualTypes[OutT, ArgT exec.FixedWidthTypes](ops 
binaryOps[OutT, ArgT, ArgT]) exec.ArrayKernelExec {

Review Comment:
   Is this useful?



##########
go/arrow/compute/internal/kernels/base_arithmetic.go:
##########
@@ -0,0 +1,141 @@
+// 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.
+
+package kernels
+
+import (
+       "fmt"
+       "math/bits"
+
+       "github.com/apache/arrow/go/v10/arrow"
+       "github.com/apache/arrow/go/v10/arrow/compute/internal/exec"
+       "github.com/apache/arrow/go/v10/arrow/internal/debug"
+       "golang.org/x/exp/constraints"
+)
+
+type ArithmeticOp int8
+
+const (
+       OpAdd ArithmeticOp = iota
+       OpAddChecked
+       OpSub
+       OpSubChecked
+)
+
+func getGoArithmeticBinaryOpsFloating[T constraints.Float](op ArithmeticOp) 
binaryOps[T, T, T] {
+       Op := map[ArithmeticOp]func(a, b T, e *error) T{
+               OpAdd:        func(a, b T, _ *error) T { return a + b },
+               OpAddChecked: func(a, b T, _ *error) T { return a + b },
+               OpSub:        func(a, b T, _ *error) T { return a - b },
+               OpSubChecked: func(a, b T, _ *error) T { return a - b },
+       }[op]
+
+       return binaryOps[T, T, T]{
+               arrArr: func(_ *exec.KernelCtx, left, right, out []T) error {
+                       var err error
+                       for i := range out {
+                               out[i] = Op(left[i], right[i], &err)
+                       }
+                       return err
+               },
+               arrScalar: func(ctx *exec.KernelCtx, left []T, right T, out 
[]T) error {
+                       var err error
+                       for i := range out {
+                               out[i] = Op(left[i], right, &err)
+                       }
+                       return err
+               },
+               scalarArr: func(ctx *exec.KernelCtx, left T, right, out []T) 
error {
+                       var err error
+                       for i := range out {
+                               out[i] = Op(left, right[i], &err)
+                       }
+                       return err
+               },
+       }
+}
+
+func getGoArithmeticBinaryOpsIntegral[T exec.UintTypes | exec.IntTypes](op 
ArithmeticOp) binaryOps[T, T, T] {
+       Op := map[ArithmeticOp]func(a, b T, e *error) T{
+               OpAdd: func(a, b T, _ *error) T { return a + b },
+               OpAddChecked: func(a, b T, e *error) T {
+                       out, carry := bits.Add64(uint64(a), uint64(b), 0)
+                       if carry > 0 {
+                               *e = fmt.Errorf("%w: overflow", 
arrow.ErrInvalid)
+                       }
+                       return T(out)
+               },
+               OpSub: func(a, b T, _ *error) T { return a - b },
+               OpSubChecked: func(a, b T, e *error) T {
+                       out, carry := bits.Sub64(uint64(a), uint64(b), 0)
+                       if carry > 0 {
+                               *e = fmt.Errorf("%w: overflow", 
arrow.ErrInvalid)
+                       }
+                       return T(out)
+               },
+       }[op]
+
+       return binaryOps[T, T, T]{
+               arrArr: func(_ *exec.KernelCtx, left, right, out []T) error {
+                       var err error
+                       for i := range out {
+                               out[i] = Op(left[i], right[i], &err)

Review Comment:
   You can't do that as it will also run overflow checks on null entries, AFAIU.



##########
go/arrow/compute/arithmetic_test.go:
##########
@@ -0,0 +1,229 @@
+// 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.
+
+package compute_test
+
+import (
+       "context"
+       "fmt"
+       "strings"
+       "testing"
+
+       "github.com/apache/arrow/go/v10/arrow"
+       "github.com/apache/arrow/go/v10/arrow/array"
+       "github.com/apache/arrow/go/v10/arrow/compute"
+       "github.com/apache/arrow/go/v10/arrow/compute/internal/exec"
+       "github.com/apache/arrow/go/v10/arrow/memory"
+       "github.com/apache/arrow/go/v10/arrow/scalar"
+       "github.com/stretchr/testify/suite"
+)
+
+type binaryFunc = func(context.Context, compute.ArithmeticOptions, 
compute.Datum, compute.Datum) (compute.Datum, error)
+
+type BinaryArithmeticSuite[T exec.NumericTypes] struct {
+       suite.Suite
+
+       mem  *memory.CheckedAllocator
+       opts compute.ArithmeticOptions
+       ctx  context.Context
+}
+
+func (BinaryArithmeticSuite[T]) DataType() arrow.DataType {
+       return exec.GetDataType[T]()
+}
+
+func (b *BinaryArithmeticSuite[T]) SetupTest() {
+       b.mem = memory.NewCheckedAllocator(memory.DefaultAllocator)
+       b.opts.CheckOverflow = false
+       b.ctx = compute.WithAllocator(context.TODO(), b.mem)
+}
+
+func (b *BinaryArithmeticSuite[T]) TearDownTest() {
+       b.mem.AssertSize(b.T(), 0)
+}
+
+func (b *BinaryArithmeticSuite[T]) makeNullScalar() scalar.Scalar {
+       return scalar.MakeNullScalar(b.DataType())
+}
+
+func (b *BinaryArithmeticSuite[T]) makeScalar(val T) scalar.Scalar {
+       return scalar.MakeScalar(val)
+}
+
+func (b *BinaryArithmeticSuite[T]) assertBinopScalars(fn binaryFunc, lhs, rhs 
T, expected T) {
+       left, right := b.makeScalar(lhs), b.makeScalar(rhs)
+       exp := b.makeScalar(expected)
+
+       actual, err := fn(b.ctx, b.opts, &compute.ScalarDatum{Value: left}, 
&compute.ScalarDatum{Value: right})
+       b.NoError(err)
+       sc := actual.(*compute.ScalarDatum).Value
+
+       b.Truef(scalar.Equals(exp, sc), "expected: %s\ngot: %s", exp, sc)
+}
+
+func (b *BinaryArithmeticSuite[T]) assertBinopScArr(fn binaryFunc, lhs T, rhs, 
expected string) {
+       left := b.makeScalar(lhs)
+       b.assertBinopScalarArr(fn, left, rhs, expected)
+}
+
+func (b *BinaryArithmeticSuite[T]) assertBinopScalarArr(fn binaryFunc, lhs 
scalar.Scalar, rhs, expected string) {
+       right, _, _ := array.FromJSON(b.mem, b.DataType(), 
strings.NewReader(rhs))
+       defer right.Release()
+       exp, _, _ := array.FromJSON(b.mem, b.DataType(), 
strings.NewReader(expected))
+       defer exp.Release()
+
+       actual, err := fn(b.ctx, b.opts, &compute.ScalarDatum{Value: lhs}, 
&compute.ArrayDatum{Value: right.Data()})
+       b.NoError(err)
+       defer actual.Release()
+       assertDatumsEqual(b.T(), &compute.ArrayDatum{Value: exp.Data()}, actual)
+}
+
+func (b *BinaryArithmeticSuite[T]) assertBinopArrSc(fn binaryFunc, lhs string, 
rhs T, expected string) {
+       right := b.makeScalar(rhs)
+       b.assertBinopArrScalar(fn, lhs, right, expected)
+}
+
+func (b *BinaryArithmeticSuite[T]) assertBinopArrScalar(fn binaryFunc, lhs 
string, rhs scalar.Scalar, expected string) {
+       left, _, _ := array.FromJSON(b.mem, b.DataType(), 
strings.NewReader(lhs))
+       defer left.Release()
+       exp, _, _ := array.FromJSON(b.mem, b.DataType(), 
strings.NewReader(expected))
+       defer exp.Release()
+
+       actual, err := fn(b.ctx, b.opts, &compute.ArrayDatum{Value: 
left.Data()}, &compute.ScalarDatum{Value: rhs})
+       b.NoError(err)
+       defer actual.Release()
+       assertDatumsEqual(b.T(), &compute.ArrayDatum{Value: exp.Data()}, actual)
+}
+
+func (b *BinaryArithmeticSuite[T]) assertBinopArrays(fn binaryFunc, lhs, rhs, 
expected string) {
+       left, _, _ := array.FromJSON(b.mem, b.DataType(), 
strings.NewReader(lhs))
+       defer left.Release()
+       right, _, _ := array.FromJSON(b.mem, b.DataType(), 
strings.NewReader(rhs))
+       defer right.Release()
+       exp, _, _ := array.FromJSON(b.mem, b.DataType(), 
strings.NewReader(expected))
+       defer exp.Release()
+
+       b.assertBinop(fn, left, right, exp)
+}
+
+func (b *BinaryArithmeticSuite[T]) assertBinop(fn binaryFunc, left, right, 
expected arrow.Array) {

Review Comment:
   Here as well, you probably want to expose this as a more general test 
helper, since this won't be useful for other things than arithmetic functions.



##########
go/arrow/compute/utils.go:
##########
@@ -81,3 +84,159 @@ func (b *bufferWriteSeeker) Seek(offset int64, whence int) 
(int64, error) {
        b.pos = newpos
        return int64(newpos), nil
 }
+
+func ensureDictionaryDecoded(vals ...arrow.DataType) {
+       for i, v := range vals {
+               if v.ID() == arrow.DICTIONARY {
+                       vals[i] = v.(*arrow.DictionaryType).ValueType

Review Comment:
   For the record, where does the actual dictionary decoding happen? (add a 
comment?)



##########
go/arrow/compute/arithmetic_test.go:
##########
@@ -0,0 +1,229 @@
+// 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.
+
+package compute_test
+
+import (
+       "context"
+       "fmt"
+       "strings"
+       "testing"
+
+       "github.com/apache/arrow/go/v10/arrow"
+       "github.com/apache/arrow/go/v10/arrow/array"
+       "github.com/apache/arrow/go/v10/arrow/compute"
+       "github.com/apache/arrow/go/v10/arrow/compute/internal/exec"
+       "github.com/apache/arrow/go/v10/arrow/memory"
+       "github.com/apache/arrow/go/v10/arrow/scalar"
+       "github.com/stretchr/testify/suite"
+)
+
+type binaryFunc = func(context.Context, compute.ArithmeticOptions, 
compute.Datum, compute.Datum) (compute.Datum, error)
+
+type BinaryArithmeticSuite[T exec.NumericTypes] struct {
+       suite.Suite
+
+       mem  *memory.CheckedAllocator
+       opts compute.ArithmeticOptions
+       ctx  context.Context
+}
+
+func (BinaryArithmeticSuite[T]) DataType() arrow.DataType {
+       return exec.GetDataType[T]()
+}
+
+func (b *BinaryArithmeticSuite[T]) SetupTest() {
+       b.mem = memory.NewCheckedAllocator(memory.DefaultAllocator)
+       b.opts.CheckOverflow = false
+       b.ctx = compute.WithAllocator(context.TODO(), b.mem)
+}
+
+func (b *BinaryArithmeticSuite[T]) TearDownTest() {
+       b.mem.AssertSize(b.T(), 0)
+}
+
+func (b *BinaryArithmeticSuite[T]) makeNullScalar() scalar.Scalar {
+       return scalar.MakeNullScalar(b.DataType())
+}
+
+func (b *BinaryArithmeticSuite[T]) makeScalar(val T) scalar.Scalar {
+       return scalar.MakeScalar(val)
+}
+
+func (b *BinaryArithmeticSuite[T]) assertBinopScalars(fn binaryFunc, lhs, rhs 
T, expected T) {
+       left, right := b.makeScalar(lhs), b.makeScalar(rhs)
+       exp := b.makeScalar(expected)
+
+       actual, err := fn(b.ctx, b.opts, &compute.ScalarDatum{Value: left}, 
&compute.ScalarDatum{Value: right})
+       b.NoError(err)
+       sc := actual.(*compute.ScalarDatum).Value
+
+       b.Truef(scalar.Equals(exp, sc), "expected: %s\ngot: %s", exp, sc)
+}
+
+func (b *BinaryArithmeticSuite[T]) assertBinopScArr(fn binaryFunc, lhs T, rhs, 
expected string) {
+       left := b.makeScalar(lhs)
+       b.assertBinopScalarArr(fn, left, rhs, expected)
+}
+
+func (b *BinaryArithmeticSuite[T]) assertBinopScalarArr(fn binaryFunc, lhs 
scalar.Scalar, rhs, expected string) {
+       right, _, _ := array.FromJSON(b.mem, b.DataType(), 
strings.NewReader(rhs))
+       defer right.Release()
+       exp, _, _ := array.FromJSON(b.mem, b.DataType(), 
strings.NewReader(expected))
+       defer exp.Release()
+
+       actual, err := fn(b.ctx, b.opts, &compute.ScalarDatum{Value: lhs}, 
&compute.ArrayDatum{Value: right.Data()})
+       b.NoError(err)
+       defer actual.Release()
+       assertDatumsEqual(b.T(), &compute.ArrayDatum{Value: exp.Data()}, actual)
+}
+
+func (b *BinaryArithmeticSuite[T]) assertBinopArrSc(fn binaryFunc, lhs string, 
rhs T, expected string) {
+       right := b.makeScalar(rhs)
+       b.assertBinopArrScalar(fn, lhs, right, expected)
+}
+
+func (b *BinaryArithmeticSuite[T]) assertBinopArrScalar(fn binaryFunc, lhs 
string, rhs scalar.Scalar, expected string) {
+       left, _, _ := array.FromJSON(b.mem, b.DataType(), 
strings.NewReader(lhs))
+       defer left.Release()
+       exp, _, _ := array.FromJSON(b.mem, b.DataType(), 
strings.NewReader(expected))
+       defer exp.Release()
+
+       actual, err := fn(b.ctx, b.opts, &compute.ArrayDatum{Value: 
left.Data()}, &compute.ScalarDatum{Value: rhs})
+       b.NoError(err)
+       defer actual.Release()
+       assertDatumsEqual(b.T(), &compute.ArrayDatum{Value: exp.Data()}, actual)
+}
+
+func (b *BinaryArithmeticSuite[T]) assertBinopArrays(fn binaryFunc, lhs, rhs, 
expected string) {
+       left, _, _ := array.FromJSON(b.mem, b.DataType(), 
strings.NewReader(lhs))
+       defer left.Release()
+       right, _, _ := array.FromJSON(b.mem, b.DataType(), 
strings.NewReader(rhs))
+       defer right.Release()
+       exp, _, _ := array.FromJSON(b.mem, b.DataType(), 
strings.NewReader(expected))
+       defer exp.Release()
+
+       b.assertBinop(fn, left, right, exp)
+}
+
+func (b *BinaryArithmeticSuite[T]) assertBinop(fn binaryFunc, left, right, 
expected arrow.Array) {
+       actual, err := fn(b.ctx, b.opts, &compute.ArrayDatum{Value: 
left.Data()}, &compute.ArrayDatum{Value: right.Data()})
+       b.Require().NoError(err)
+       defer actual.Release()
+       assertDatumsEqual(b.T(), &compute.ArrayDatum{Value: expected.Data()}, 
actual)
+
+       // also check (Scalar, Scalar) operations
+       for i := 0; i < expected.Len(); i++ {
+               s, err := scalar.GetScalar(expected, i)
+               b.Require().NoError(err)
+               lhs, _ := scalar.GetScalar(left, i)
+               rhs, _ := scalar.GetScalar(right, i)
+
+               actual, err := fn(b.ctx, b.opts, &compute.ScalarDatum{Value: 
lhs}, &compute.ScalarDatum{Value: rhs})
+               b.NoError(err)
+               b.Truef(scalar.Equals(s, actual.(*compute.ScalarDatum).Value), 
"expected: %s\ngot: %s", s, actual)
+       }
+}
+
+func (b *BinaryArithmeticSuite[T]) setOverflowCheck(value bool) {
+       b.opts.CheckOverflow = value
+}
+
+func (b *BinaryArithmeticSuite[T]) assertBinopErr(fn binaryFunc, lhs, rhs, 
expectedMsg string) {
+       left, _, _ := array.FromJSON(b.mem, b.DataType(), 
strings.NewReader(lhs))
+       defer left.Release()
+       right, _, _ := array.FromJSON(b.mem, b.DataType(), 
strings.NewReader(rhs))
+       defer right.Release()
+
+       _, err := fn(b.ctx, b.opts, &compute.ArrayDatum{left.Data()}, 
&compute.ArrayDatum{Value: right.Data()})
+       b.ErrorIs(err, arrow.ErrInvalid)
+       b.ErrorContains(err, expectedMsg)
+}
+
+func (b *BinaryArithmeticSuite[T]) TestAdd() {
+       b.Run(b.DataType().String(), func() {
+               for _, overflow := range []bool{false, true} {
+                       b.Run(fmt.Sprintf("overflow=%t", overflow), func() {
+                               b.setOverflowCheck(overflow)
+
+                               b.assertBinopArrays(compute.Add, `[]`, `[]`, 
`[]`)
+                               b.assertBinopArrays(compute.Add, `[3, 2, 6]`, 
`[1, 0, 2]`, `[4, 2, 8]`)
+                               // nulls on one side
+                               b.assertBinopArrays(compute.Add, `[null, 1, 
null]`, `[3, 4, 5]`, `[null, 5, null]`)
+                               b.assertBinopArrays(compute.Add, `[3, 4, 5]`, 
`[null, 1, null]`, `[null, 5, null]`)
+                               // nulls on both sides
+                               b.assertBinopArrays(compute.Add, `[null, 1, 
2]`, `[3, 4, null]`, `[null, 5, null]`)
+                               // all nulls
+                               b.assertBinopArrays(compute.Add, `[null]`, 
`[null]`, `[null]`)
+
+                               // scalar on the left
+                               b.assertBinopScArr(compute.Add, 3, `[1, 2]`, 
`[4, 5]`)
+                               b.assertBinopScArr(compute.Add, 3, `[null, 2]`, 
`[null, 5]`)
+                               b.assertBinopScalarArr(compute.Add, 
b.makeNullScalar(), `[1, 2]`, `[null, null]`)
+                               b.assertBinopScalarArr(compute.Add, 
b.makeNullScalar(), `[null, 2]`, `[null, null]`)
+                               // scalar on the right
+                               b.assertBinopArrSc(compute.Add, `[1, 2]`, 3, 
`[4, 5]`)
+                               b.assertBinopArrSc(compute.Add, `[null, 2]`, 3, 
`[null, 5]`)
+                               b.assertBinopArrScalar(compute.Add, `[1, 2]`, 
b.makeNullScalar(), `[null, null]`)
+                               b.assertBinopArrScalar(compute.Add, `[null, 
2]`, b.makeNullScalar(), `[null, null]`)
+                       })
+               }
+       })
+}
+
+func (b *BinaryArithmeticSuite[T]) TestSub() {
+       b.Run(b.DataType().String(), func() {
+               for _, overflow := range []bool{false, true} {
+                       b.Run(fmt.Sprintf("overflow=%t", overflow), func() {
+                               b.setOverflowCheck(overflow)
+
+                               b.assertBinopArrays(compute.Subtract, `[]`, 
`[]`, `[]`)
+                               b.assertBinopArrays(compute.Subtract, `[3, 2, 
6]`, `[1, 0, 2]`, `[2, 2, 4]`)
+                               // nulls on one side
+                               b.assertBinopArrays(compute.Subtract, `[null, 
4, null]`, `[2, 1, 0]`, `[null, 3, null]`)
+                               b.assertBinopArrays(compute.Subtract, `[3, 4, 
5]`, `[null, 1, null]`, `[null, 3, null]`)
+                               // nulls on both sides
+                               b.assertBinopArrays(compute.Subtract, `[null, 
4, 3]`, `[2, 1, null]`, `[null, 3, null]`)
+                               // all nulls
+                               b.assertBinopArrays(compute.Subtract, `[null]`, 
`[null]`, `[null]`)
+
+                               // scalar on the left
+                               b.assertBinopScArr(compute.Subtract, 3, `[1, 
2]`, `[2, 1]`)
+                               b.assertBinopScArr(compute.Subtract, 3, `[null, 
2]`, `[null, 1]`)
+                               b.assertBinopScalarArr(compute.Subtract, 
b.makeNullScalar(), `[1, 2]`, `[null, null]`)
+                               b.assertBinopScalarArr(compute.Subtract, 
b.makeNullScalar(), `[null, 2]`, `[null, null]`)
+                               // scalar on the right
+                               b.assertBinopArrSc(compute.Subtract, `[4, 5]`, 
3, `[1, 2]`)
+                               b.assertBinopArrSc(compute.Subtract, `[null, 
5]`, 3, `[null, 2]`)
+                               b.assertBinopArrScalar(compute.Subtract, `[1, 
2]`, b.makeNullScalar(), `[null, null]`)
+                               b.assertBinopArrScalar(compute.Subtract, 
`[null, 2]`, b.makeNullScalar(), `[null, null]`)
+                       })
+               }
+       })
+}
+
+func TestBinaryArithmetic(t *testing.T) {
+       suite.Run(t, new(BinaryArithmeticSuite[int8]))
+       suite.Run(t, new(BinaryArithmeticSuite[uint8]))
+       suite.Run(t, new(BinaryArithmeticSuite[int16]))
+       suite.Run(t, new(BinaryArithmeticSuite[uint16]))
+       suite.Run(t, new(BinaryArithmeticSuite[int32]))
+       suite.Run(t, new(BinaryArithmeticSuite[uint32]))
+       suite.Run(t, new(BinaryArithmeticSuite[int64]))
+       suite.Run(t, new(BinaryArithmeticSuite[uint64]))
+       suite.Run(t, new(BinaryArithmeticSuite[float32]))
+       suite.Run(t, new(BinaryArithmeticSuite[float64]))

Review Comment:
   Is float16 supported? Regardless of the answer, add a test for it?



##########
go/arrow/compute/utils.go:
##########
@@ -81,3 +84,159 @@ func (b *bufferWriteSeeker) Seek(offset int64, whence int) 
(int64, error) {
        b.pos = newpos
        return int64(newpos), nil
 }
+
+func ensureDictionaryDecoded(vals ...arrow.DataType) {

Review Comment:
   Can you add docstrings to all these helpers?



##########
go/arrow/compute/internal/kernels/base_arithmetic_amd64.go:
##########
@@ -0,0 +1,83 @@
+// 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.
+
+//go:build !noasm
+
+package kernels
+
+import (
+       "unsafe"
+
+       "github.com/apache/arrow/go/v10/arrow/compute/internal/exec"
+       "golang.org/x/exp/constraints"
+       "golang.org/x/sys/cpu"
+)
+
+func getAvx2ArithmeticBinaryNumeric[T exec.NumericTypes](op ArithmeticOp) 
binaryOps[T, T, T] {
+       typ := exec.GetType[T]()
+       return binaryOps[T, T, T]{
+               arrArr: func(_ *exec.KernelCtx, Arg0, Arg1, Out []T) error {
+                       arithmeticAvx2(typ, op, exec.GetBytes(Arg0), 
exec.GetBytes(Arg1), exec.GetBytes(Out), len(Out))
+                       return nil
+               },
+               arrScalar: func(_ *exec.KernelCtx, Arg0 []T, Arg1 T, Out []T) 
error {
+                       arithmeticArrScalarAvx2(typ, op, exec.GetBytes(Arg0), 
unsafe.Pointer(&Arg1), exec.GetBytes(Out), len(Out))
+                       return nil
+               },
+               scalarArr: func(_ *exec.KernelCtx, Arg0 T, Arg1, Out []T) error 
{
+                       arithmeticScalarArrAvx2(typ, op, unsafe.Pointer(&Arg0), 
exec.GetBytes(Arg1), exec.GetBytes(Out), len(Out))
+                       return nil
+               },
+       }
+}
+
+func getSSE4ArithmeticBinaryNumeric[T exec.NumericTypes](op ArithmeticOp) 
binaryOps[T, T, T] {
+       typ := exec.GetType[T]()
+       return binaryOps[T, T, T]{
+               arrArr: func(_ *exec.KernelCtx, Arg0, Arg1, Out []T) error {
+                       arithmeticSSE4(typ, op, exec.GetBytes(Arg0), 
exec.GetBytes(Arg1), exec.GetBytes(Out), len(Out))
+                       return nil
+               },
+               arrScalar: func(_ *exec.KernelCtx, Arg0 []T, Arg1 T, Out []T) 
error {
+                       arithmeticArrScalarSSE4(typ, op, exec.GetBytes(Arg0), 
unsafe.Pointer(&Arg1), exec.GetBytes(Out), len(Out))
+                       return nil
+               },
+               scalarArr: func(_ *exec.KernelCtx, Arg0 T, Arg1, Out []T) error 
{
+                       arithmeticScalarArrSSE4(typ, op, unsafe.Pointer(&Arg0), 
exec.GetBytes(Arg1), exec.GetBytes(Out), len(Out))
+                       return nil
+               },
+       }
+}
+
+func getArithmeticBinaryOpsFloating[T constraints.Float](op ArithmeticOp) 
binaryOps[T, T, T] {
+       if cpu.X86.HasAVX2 {
+               return getAvx2ArithmeticBinaryNumeric[T](op)
+       } else if cpu.X86.HasSSE42 {
+               return getSSE4ArithmeticBinaryNumeric[T](op)
+       }
+
+       return getGoArithmeticBinaryOpsFloating[T](op)

Review Comment:
   I'm curious, why not also compile this one from C++? 1) the C++ optimizer is 
probably better than the Go one, 2) less code to write and maintain?



##########
go/arrow/compute/internal/exec/utils.go:
##########
@@ -172,6 +179,11 @@ func GetDataType[T NumericTypes | bool | string]() 
arrow.DataType {
        return typMap[reflect.TypeOf(z)]
 }
 
+func GetType[T NumericTypes | bool | string]() arrow.Type {

Review Comment:
   Also why is it not defined for all types?



##########
go/arrow/compute/internal/kernels/_lib/base_arithmetic.cc:
##########
@@ -0,0 +1,255 @@
+// 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.
+
+#include <arch.h>
+#include <stdint.h>
+#include "types.h"
+#include "vendored/safe-math.h"
+
+// Define functions AddWithOverflow, SubtractWithOverflow, MultiplyWithOverflow
+// with the signature `bool(T u, T v, T* out)` where T is an integer type.
+// On overflow, these functions return true.  Otherwise, false is returned
+// and `out` is updated with the result of the operation.
+
+#define OP_WITH_OVERFLOW(_func_name, _psnip_op, _type, _psnip_type) \
+  static inline bool _func_name(_type u, _type v, _type* out) {     \
+    return !psnip_safe_##_psnip_type##_##_psnip_op(out, u, v);      \
+  }
+
+#define OPS_WITH_OVERFLOW(_func_name, _psnip_op)            \
+  OP_WITH_OVERFLOW(_func_name, _psnip_op, int8_t, int8)     \
+  OP_WITH_OVERFLOW(_func_name, _psnip_op, int16_t, int16)   \
+  OP_WITH_OVERFLOW(_func_name, _psnip_op, int32_t, int32)   \
+  OP_WITH_OVERFLOW(_func_name, _psnip_op, int64_t, int64)   \
+  OP_WITH_OVERFLOW(_func_name, _psnip_op, uint8_t, uint8)   \
+  OP_WITH_OVERFLOW(_func_name, _psnip_op, uint16_t, uint16) \
+  OP_WITH_OVERFLOW(_func_name, _psnip_op, uint32_t, uint32) \
+  OP_WITH_OVERFLOW(_func_name, _psnip_op, uint64_t, uint64)
+
+OPS_WITH_OVERFLOW(AddWithOverflow, add)
+OPS_WITH_OVERFLOW(SubtractWithOverflow, sub)
+OPS_WITH_OVERFLOW(MultiplyWithOverflow, mul)
+OPS_WITH_OVERFLOW(DivideWithOverflow, div)
+
+// Corresponds to equivalent ArithmeticOp enum in base_arithmetic.go
+// for passing across which operation to perform. This allows simpler
+// implementation at the cost of having to pass the extra int8 and
+// perform a switch.
+//
+// In cases of small arrays, this is completely negligible. In cases
+// of large arrays, the time saved by using SIMD here is significantly
+// worth the cost.
+enum class optype : int8_t {
+    ADD,
+    ADD_CHECKED,
+    SUB, 
+    SUB_CHECKED,
+};
+
+template <typename T>
+using is_unsigned_integer_value = bool_constant<is_integral_v<T> && 
is_unsigned_v<T>>;
+
+template <typename T>
+using is_signed_integer_value = bool_constant<is_integral_v<T> && 
is_signed_v<T>>;
+
+template <typename T, typename R = T>
+using enable_if_signed_integer_t = 
enable_if_t<is_signed_integer_value<T>::value, R>;
+
+template <typename T, typename R = T>
+using enable_if_unsigned_integer_t = 
enable_if_t<is_unsigned_integer_value<T>::value, R>;
+
+template <typename T, typename R = T>
+using enable_if_integer_t = enable_if_t<
+    is_signed_integer_value<T>::value || is_unsigned_integer_value<T>::value, 
R>;
+
+template <typename T, typename R = T>
+using enable_if_floating_t = enable_if_t<is_floating_point_v<T>, R>;
+
+struct Add {
+    template <typename T, typename Arg0, typename Arg1>
+    static constexpr enable_if_floating_t<T> Call(Arg0 left, Arg1 right, 
bool*) {
+        return left + right;
+    }
+
+    template <typename T, typename Arg0, typename Arg1>
+    static constexpr enable_if_integer_t<T> Call(Arg0 left, Arg1 right, bool*) 
{
+        return left + right;
+    }
+};
+
+struct Sub {
+    template <typename T, typename Arg0, typename Arg1>
+    static constexpr enable_if_floating_t<T> Call(Arg0 left, Arg1 right, 
bool*) {
+        return left - right;
+    }
+
+    template <typename T, typename Arg0, typename Arg1>
+    static constexpr enable_if_integer_t<T> Call(Arg0 left, Arg1 right, bool*) 
{
+        return left - right;
+    }
+};
+
+struct AddChecked {
+    template <typename T, typename Arg0, typename Arg1>
+    static constexpr enable_if_floating_t<T> Call(Arg0 left, Arg1 right, 
bool*) {
+        return left + right;
+    }
+
+    template <typename T, typename Arg0, typename Arg1>
+    static constexpr enable_if_integer_t<T> Call(Arg0 left, Arg1 right, bool* 
failure) {

Review Comment:
   For the record, this would probably be:
   ```c++
   struct AddChecked {
       template <typename T, typename Arg0, typename Arg1>
       static constexpr void Call(Arg0 left, Arg1 right, bool* failure) {
           static_assert(is_same<T, Arg0>::value && is_same<T, Arg1>::value, 
"");
           if constexpr(is_signed_integer_value<T>::value ||
                        is_unsigned_integer_value<T>::value) {
               T result = 0;
               if (AddWithOverflow(left, right, &result)) {
                   *failure = true;
               }
               return result;
           } else {
               return left + right;
           }
       }
   };
   ```



##########
go/arrow/compute/utils.go:
##########
@@ -81,3 +84,159 @@ func (b *bufferWriteSeeker) Seek(offset int64, whence int) 
(int64, error) {
        b.pos = newpos
        return int64(newpos), nil
 }
+
+func ensureDictionaryDecoded(vals ...arrow.DataType) {

Review Comment:
   Also, are there any tests for them?



##########
go/arrow/compute/internal/kernels/_lib/types.h:
##########
@@ -0,0 +1,477 @@
+// 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.
+
+// corresponds to datatype.go's arrow.Type
+enum class arrtype : int {
+    NULL,
+    BOOL,
+    UINT8,
+    INT8,
+    UINT16,
+    INT16,
+    UINT32,
+    INT32,
+    UINT64,
+    INT64,
+    FLOAT16,
+    FLOAT32,
+    FLOAT64
+};
+
+// The following is copied from <type_traits> since we use -target 

Review Comment:
   If this is from libstdc++, then it's under the GPLv3 and cannot be copied 
here.
   



##########
go/arrow/compute/arithmetic.go:
##########
@@ -0,0 +1,155 @@
+// 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.
+
+package compute
+
+import (
+       "context"
+       "fmt"
+       "strings"
+
+       "github.com/apache/arrow/go/v10/arrow"
+       "github.com/apache/arrow/go/v10/arrow/compute/internal/exec"
+       "github.com/apache/arrow/go/v10/arrow/compute/internal/kernels"
+)
+
+type arithmeticFunction struct {
+       ScalarFunction
+}
+
+func (fn *arithmeticFunction) checkDecimals(vals ...arrow.DataType) error {
+       if !hasDecimal(vals...) {
+               return nil
+       }
+
+       if len(vals) != 2 {
+               return nil
+       }
+
+       op := fn.name[:strings.Index(fn.name, "_")]
+       switch op {
+       case "add", "subtract":
+               return castBinaryDecimalArgs(decPromoteAdd, vals...)
+       case "multiply":
+               return castBinaryDecimalArgs(decPromoteMultiply, vals...)
+       case "divide":
+               return castBinaryDecimalArgs(decPromoteDivide, vals...)
+       default:
+               return fmt.Errorf("%w: invalid decimal function: %s", 
arrow.ErrInvalid, fn.name)
+       }
+}
+
+func (fn *arithmeticFunction) DispatchBest(vals ...arrow.DataType) 
(exec.Kernel, error) {
+       if err := fn.checkArity(len(vals)); err != nil {
+               return nil, err
+       }
+
+       if err := fn.checkDecimals(vals...); err != nil {
+               return nil, err
+       }
+
+       if kn, err := fn.DispatchExact(vals...); err == nil {
+               return kn, nil
+       }
+
+       ensureDictionaryDecoded(vals...)
+
+       // only promote types for binary funcs
+       if len(vals) == 2 {
+               replaceNullWithOtherType(vals...)
+               if unit, istime := commonTemporalResolution(vals...); istime {
+                       replaceTemporalTypes(unit, vals...)
+               } else {
+                       if dt := commonNumeric(vals...); dt != nil {
+                               replaceTypes(dt, vals...)
+                       }
+               }
+       }
+
+       return fn.DispatchExact(vals...)
+}
+
+var (
+       addDoc FunctionDoc
+)
+
+func RegisterScalarArithmetic(reg FunctionRegistry) {
+       addFn := &arithmeticFunction{*NewScalarFunction("add", Binary(), 
addDoc)}
+       for _, k := range kernels.GetArithmeticKernels(kernels.OpAdd) {
+               if err := addFn.AddKernel(k); err != nil {
+                       panic(err)
+               }
+       }
+
+       reg.AddFunction(addFn, false)
+
+       addCheckedFn := &arithmeticFunction{*NewScalarFunction("add_checked", 
Binary(), addDoc)}

Review Comment:
   (I'm not even sure unchecked arithmetic is worth adding TBH)



##########
go/arrow/compute/internal/kernels/base_arithmetic.go:
##########
@@ -0,0 +1,141 @@
+// 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.
+
+package kernels
+
+import (
+       "fmt"
+       "math/bits"
+
+       "github.com/apache/arrow/go/v10/arrow"
+       "github.com/apache/arrow/go/v10/arrow/compute/internal/exec"
+       "github.com/apache/arrow/go/v10/arrow/internal/debug"
+       "golang.org/x/exp/constraints"
+)
+
+type ArithmeticOp int8
+
+const (
+       OpAdd ArithmeticOp = iota
+       OpAddChecked
+       OpSub
+       OpSubChecked
+)
+
+func getGoArithmeticBinaryOpsFloating[T constraints.Float](op ArithmeticOp) 
binaryOps[T, T, T] {
+       Op := map[ArithmeticOp]func(a, b T, e *error) T{

Review Comment:
   Why pass an error pointer that never gets used?



##########
go/arrow/compute/internal/kernels/base_arithmetic.go:
##########
@@ -0,0 +1,141 @@
+// 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.
+
+package kernels
+
+import (
+       "fmt"
+       "math/bits"
+
+       "github.com/apache/arrow/go/v10/arrow"
+       "github.com/apache/arrow/go/v10/arrow/compute/internal/exec"
+       "github.com/apache/arrow/go/v10/arrow/internal/debug"
+       "golang.org/x/exp/constraints"
+)
+
+type ArithmeticOp int8
+
+const (
+       OpAdd ArithmeticOp = iota
+       OpAddChecked
+       OpSub
+       OpSubChecked
+)
+
+func getGoArithmeticBinaryOpsFloating[T constraints.Float](op ArithmeticOp) 
binaryOps[T, T, T] {
+       Op := map[ArithmeticOp]func(a, b T, e *error) T{
+               OpAdd:        func(a, b T, _ *error) T { return a + b },
+               OpAddChecked: func(a, b T, _ *error) T { return a + b },
+               OpSub:        func(a, b T, _ *error) T { return a - b },
+               OpSubChecked: func(a, b T, _ *error) T { return a - b },
+       }[op]
+
+       return binaryOps[T, T, T]{
+               arrArr: func(_ *exec.KernelCtx, left, right, out []T) error {
+                       var err error
+                       for i := range out {
+                               out[i] = Op(left[i], right[i], &err)

Review Comment:
   Does this actually call a runtime-selected function pointer on every loop 
iteration?



##########
go/arrow/compute/internal/exec/utils.go:
##########
@@ -135,6 +135,13 @@ func Min[T constraints.Ordered](a, b T) T {
        return b
 }
 
+func Max[T constraints.Ordered](a, b T) T {

Review Comment:
   Do you want this to be well-behaved for signed zeros? i.e. do `Max(0.0, 
-0.0)` and `Max(-0.0, 0.0)` both return 0.0?



##########
go/arrow/compute/internal/exec/utils.go:
##########
@@ -172,6 +179,11 @@ func GetDataType[T NumericTypes | bool | string]() 
arrow.DataType {
        return typMap[reflect.TypeOf(z)]
 }
 
+func GetType[T NumericTypes | bool | string]() arrow.Type {

Review Comment:
   What does this do? Add a docstring?



##########
go/arrow/compute/arithmetic_test.go:
##########
@@ -0,0 +1,229 @@
+// 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.
+
+package compute_test
+
+import (
+       "context"
+       "fmt"
+       "strings"
+       "testing"
+
+       "github.com/apache/arrow/go/v10/arrow"
+       "github.com/apache/arrow/go/v10/arrow/array"
+       "github.com/apache/arrow/go/v10/arrow/compute"
+       "github.com/apache/arrow/go/v10/arrow/compute/internal/exec"
+       "github.com/apache/arrow/go/v10/arrow/memory"
+       "github.com/apache/arrow/go/v10/arrow/scalar"
+       "github.com/stretchr/testify/suite"
+)
+
+type binaryFunc = func(context.Context, compute.ArithmeticOptions, 
compute.Datum, compute.Datum) (compute.Datum, error)
+
+type BinaryArithmeticSuite[T exec.NumericTypes] struct {

Review Comment:
   Wow, Go has generics finally? :-)



##########
go/arrow/compute/arithmetic_test.go:
##########
@@ -0,0 +1,229 @@
+// 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.
+
+package compute_test
+
+import (
+       "context"
+       "fmt"
+       "strings"
+       "testing"
+
+       "github.com/apache/arrow/go/v10/arrow"
+       "github.com/apache/arrow/go/v10/arrow/array"
+       "github.com/apache/arrow/go/v10/arrow/compute"
+       "github.com/apache/arrow/go/v10/arrow/compute/internal/exec"
+       "github.com/apache/arrow/go/v10/arrow/memory"
+       "github.com/apache/arrow/go/v10/arrow/scalar"
+       "github.com/stretchr/testify/suite"
+)
+
+type binaryFunc = func(context.Context, compute.ArithmeticOptions, 
compute.Datum, compute.Datum) (compute.Datum, error)
+
+type BinaryArithmeticSuite[T exec.NumericTypes] struct {
+       suite.Suite
+
+       mem  *memory.CheckedAllocator
+       opts compute.ArithmeticOptions
+       ctx  context.Context
+}
+
+func (BinaryArithmeticSuite[T]) DataType() arrow.DataType {
+       return exec.GetDataType[T]()
+}
+
+func (b *BinaryArithmeticSuite[T]) SetupTest() {
+       b.mem = memory.NewCheckedAllocator(memory.DefaultAllocator)
+       b.opts.CheckOverflow = false
+       b.ctx = compute.WithAllocator(context.TODO(), b.mem)
+}
+
+func (b *BinaryArithmeticSuite[T]) TearDownTest() {
+       b.mem.AssertSize(b.T(), 0)
+}
+
+func (b *BinaryArithmeticSuite[T]) makeNullScalar() scalar.Scalar {
+       return scalar.MakeNullScalar(b.DataType())
+}
+
+func (b *BinaryArithmeticSuite[T]) makeScalar(val T) scalar.Scalar {
+       return scalar.MakeScalar(val)
+}
+
+func (b *BinaryArithmeticSuite[T]) assertBinopScalars(fn binaryFunc, lhs, rhs 
T, expected T) {
+       left, right := b.makeScalar(lhs), b.makeScalar(rhs)
+       exp := b.makeScalar(expected)
+
+       actual, err := fn(b.ctx, b.opts, &compute.ScalarDatum{Value: left}, 
&compute.ScalarDatum{Value: right})
+       b.NoError(err)
+       sc := actual.(*compute.ScalarDatum).Value
+
+       b.Truef(scalar.Equals(exp, sc), "expected: %s\ngot: %s", exp, sc)
+}
+
+func (b *BinaryArithmeticSuite[T]) assertBinopScArr(fn binaryFunc, lhs T, rhs, 
expected string) {
+       left := b.makeScalar(lhs)
+       b.assertBinopScalarArr(fn, left, rhs, expected)
+}
+
+func (b *BinaryArithmeticSuite[T]) assertBinopScalarArr(fn binaryFunc, lhs 
scalar.Scalar, rhs, expected string) {
+       right, _, _ := array.FromJSON(b.mem, b.DataType(), 
strings.NewReader(rhs))
+       defer right.Release()
+       exp, _, _ := array.FromJSON(b.mem, b.DataType(), 
strings.NewReader(expected))
+       defer exp.Release()
+
+       actual, err := fn(b.ctx, b.opts, &compute.ScalarDatum{Value: lhs}, 
&compute.ArrayDatum{Value: right.Data()})
+       b.NoError(err)
+       defer actual.Release()
+       assertDatumsEqual(b.T(), &compute.ArrayDatum{Value: exp.Data()}, actual)
+}
+
+func (b *BinaryArithmeticSuite[T]) assertBinopArrSc(fn binaryFunc, lhs string, 
rhs T, expected string) {
+       right := b.makeScalar(rhs)
+       b.assertBinopArrScalar(fn, lhs, right, expected)
+}
+
+func (b *BinaryArithmeticSuite[T]) assertBinopArrScalar(fn binaryFunc, lhs 
string, rhs scalar.Scalar, expected string) {
+       left, _, _ := array.FromJSON(b.mem, b.DataType(), 
strings.NewReader(lhs))
+       defer left.Release()
+       exp, _, _ := array.FromJSON(b.mem, b.DataType(), 
strings.NewReader(expected))
+       defer exp.Release()
+
+       actual, err := fn(b.ctx, b.opts, &compute.ArrayDatum{Value: 
left.Data()}, &compute.ScalarDatum{Value: rhs})
+       b.NoError(err)
+       defer actual.Release()
+       assertDatumsEqual(b.T(), &compute.ArrayDatum{Value: exp.Data()}, actual)
+}
+
+func (b *BinaryArithmeticSuite[T]) assertBinopArrays(fn binaryFunc, lhs, rhs, 
expected string) {
+       left, _, _ := array.FromJSON(b.mem, b.DataType(), 
strings.NewReader(lhs))
+       defer left.Release()
+       right, _, _ := array.FromJSON(b.mem, b.DataType(), 
strings.NewReader(rhs))
+       defer right.Release()
+       exp, _, _ := array.FromJSON(b.mem, b.DataType(), 
strings.NewReader(expected))
+       defer exp.Release()
+
+       b.assertBinop(fn, left, right, exp)
+}
+
+func (b *BinaryArithmeticSuite[T]) assertBinop(fn binaryFunc, left, right, 
expected arrow.Array) {
+       actual, err := fn(b.ctx, b.opts, &compute.ArrayDatum{Value: 
left.Data()}, &compute.ArrayDatum{Value: right.Data()})
+       b.Require().NoError(err)
+       defer actual.Release()
+       assertDatumsEqual(b.T(), &compute.ArrayDatum{Value: expected.Data()}, 
actual)
+
+       // also check (Scalar, Scalar) operations

Review Comment:
   Also may want to check (Array, Scalar) and (Scalar, Array) automatically?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to