diff --git a/go.work b/go.work new file mode 100644 index 0000000000000..ccfe97d5510a4 --- /dev/null +++ b/go.work @@ -0,0 +1,23 @@ +// 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 1.18 + +use ( + ./go + ./go/arrow/compute +) diff --git a/go/arrow/bitutil/bitmaps.go b/go/arrow/bitutil/bitmaps.go index 460f4924cb4a9..10fa02797797a 100644 --- a/go/arrow/bitutil/bitmaps.go +++ b/go/arrow/bitutil/bitmaps.go @@ -17,6 +17,7 @@ package bitutil import ( + "bytes" "math/bits" "unsafe" @@ -530,3 +531,45 @@ func BitmapAndAlloc(mem memory.Allocator, left, right []byte, lOffset, rOffset i func BitmapOrAlloc(mem memory.Allocator, left, right []byte, lOffset, rOffset int64, length, outOffset int64) *memory.Buffer { return BitmapOpAlloc(mem, bitOrOp, left, right, lOffset, rOffset, length, outOffset) } + +func BitmapEquals(left, right []byte, lOffset, rOffset int64, length int64) bool { + if lOffset%8 == 0 && rOffset%8 == 0 { + // byte aligned, fast path, can use bytes.Equal (memcmp) + byteLen := length / 8 + lStart := lOffset / 8 + rStart := rOffset / 8 + if !bytes.Equal(left[lStart:lStart+byteLen], right[rStart:rStart+byteLen]) { + return false + } + + // check trailing bits + for i := (length / 8) * 8; i < length; i++ { + if BitIsSet(left, int(lOffset+i)) != BitIsSet(right, int(rOffset+i)) { + return false + } + } + return true + } + + lrdr := NewBitmapWordReader(left, int(lOffset), int(length)) + rrdr := NewBitmapWordReader(right, int(rOffset), int(length)) + + nwords := lrdr.Words() + for nwords > 0 { + nwords-- + if lrdr.NextWord() != rrdr.NextWord() { + return false + } + } + + nbytes := lrdr.TrailingBytes() + for nbytes > 0 { + nbytes-- + lbt, _ := lrdr.NextTrailingByte() + rbt, _ := rrdr.NextTrailingByte() + if lbt != rbt { + return false + } + } + return true +} diff --git a/go/arrow/compute/datum.go b/go/arrow/compute/datum.go index 31c02ce4f826a..a73902e55ad97 100644 --- a/go/arrow/compute/datum.go +++ b/go/arrow/compute/datum.go @@ -30,17 +30,27 @@ import ( type DatumKind int const ( - KindNone DatumKind = iota // none - KindScalar // scalar - KindArray // array - KindChunked // chunked_array - KindRecord // record_batch - KindTable // table - KindCollection // collection + KindNone DatumKind = iota // none + KindScalar // scalar + KindArray // array + KindChunked // chunked_array + KindRecord // record_batch + KindTable // table ) const UnknownLength int64 = -1 +// DatumIsValue returns true if the datum passed is a Scalar, Array +// or ChunkedArray type (e.g. it contains a specific value not a +// group of values) +func DatumIsValue(d Datum) bool { + switch d.Kind() { + case KindScalar, KindArray, KindChunked: + return true + } + return false +} + // Datum is a variant interface for wrapping the various Arrow data structures // for now the various Datum types just hold a Value which is the type they // are wrapping, but it might make sense in the future for those types @@ -247,6 +257,9 @@ func NewDatum(value interface{}) Datum { case arrow.Array: v.Data().Retain() return &ArrayDatum{v.Data().(*array.Data)} + case arrow.ArrayData: + v.Retain() + return &ArrayDatum{v} case *arrow.Chunked: v.Retain() return &ChunkedDatum{v} diff --git a/go/arrow/compute/datumkind_string.go b/go/arrow/compute/datumkind_string.go index 56cef315ac62f..8537c0b7efe31 100644 --- a/go/arrow/compute/datumkind_string.go +++ b/go/arrow/compute/datumkind_string.go @@ -14,12 +14,11 @@ func _() { _ = x[KindChunked-3] _ = x[KindRecord-4] _ = x[KindTable-5] - _ = x[KindCollection-6] } -const _DatumKind_name = "nonescalararraychunked_arrayrecord_batchtablecollection" +const _DatumKind_name = "nonescalararraychunked_arrayrecord_batchtable" -var _DatumKind_index = [...]uint8{0, 4, 10, 15, 28, 40, 45, 55} +var _DatumKind_index = [...]uint8{0, 4, 10, 15, 28, 40, 45} func (i DatumKind) String() string { if i < 0 || i >= DatumKind(len(_DatumKind_index)-1) { diff --git a/go/arrow/compute/exec.go b/go/arrow/compute/exec.go new file mode 100644 index 0000000000000..5719ee153da5a --- /dev/null +++ b/go/arrow/compute/exec.go @@ -0,0 +1,165 @@ +// 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" + + "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" +) + +func haveChunkedArray(values []Datum) bool { + for _, v := range values { + if v.Kind() == KindChunked { + return true + } + } + return false +} + +// ExecSpanFromBatch constructs and returns a new ExecSpan from the values +// inside of the ExecBatch which could be scalar or arrays. +// +// This is mostly used for tests but is also a convenience method for other +// cases. +func ExecSpanFromBatch(batch *ExecBatch) *exec.ExecSpan { + out := &exec.ExecSpan{Len: batch.Len, Values: make([]exec.ExecValue, len(batch.Values))} + for i, v := range batch.Values { + outVal := &out.Values[i] + if v.Kind() == KindScalar { + outVal.Scalar = v.(*ScalarDatum).Value + } else { + outVal.Array.SetMembers(v.(*ArrayDatum).Value) + outVal.Scalar = nil + } + } + return out +} + +// this is the primary driver of execution +func execInternal(ctx context.Context, fn Function, opts FunctionOptions, passedLen int64, args ...Datum) (result Datum, err error) { + if opts == nil { + if err = checkOptions(fn, opts); err != nil { + return + } + opts = fn.DefaultOptions() + } + + // we only allow Array, ChunkedArray, and Scalars for now. + // RecordBatch and Table datums are disallowed. + if err = checkAllIsValue(args); err != nil { + return + } + + inTypes := make([]arrow.DataType, len(args)) + for i, a := range args { + inTypes[i] = a.(ArrayLikeDatum).Type() + } + + var ( + k exec.Kernel + executor kernelExecutor + ) + + switch fn.Kind() { + case FuncScalar: + executor = scalarExecPool.Get().(*scalarExecutor) + defer func() { + executor.clear() + scalarExecPool.Put(executor.(*scalarExecutor)) + }() + default: + return nil, fmt.Errorf("%w: direct execution of %s", arrow.ErrNotImplemented, fn.Kind()) + } + + if k, err = fn.DispatchBest(inTypes...); err != nil { + return + } + + kctx := &exec.KernelCtx{Ctx: ctx, Kernel: k} + init := k.GetInitFn() + kinitArgs := exec.KernelInitArgs{Kernel: k, Inputs: inTypes, Options: opts} + if init != nil { + kctx.State, err = init(kctx, kinitArgs) + if err != nil { + return + } + } + + if err = executor.Init(kctx, kinitArgs); err != nil { + return + } + + input := ExecBatch{Values: args, Len: 0} + if input.NumValues() == 0 { + if passedLen != -1 { + input.Len = passedLen + } + } else { + inferred, _ := inferBatchLength(input.Values) + input.Len = inferred + switch fn.Kind() { + case FuncScalar: + if passedLen != -1 && passedLen != inferred { + return nil, fmt.Errorf("%w: passed batch length for execution did not match actual length for scalar fn execution", + arrow.ErrInvalid) + } + } + } + + ectx := GetExecCtx(ctx) + + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + ch := make(chan Datum, ectx.ExecChannelSize) + go func() { + defer close(ch) + if err = executor.Execute(ctx, &input, ch); err != nil { + cancel() + } + }() + + result = executor.WrapResults(ctx, ch, haveChunkedArray(input.Values)) + debug.Assert(executor.CheckResultType(result) == nil, "invalid result type") + + if ctx.Err() == context.Canceled { + result.Release() + } + + return +} + +// CallFunction is a one-shot invoker for all types of functions. +// +// It will perform kernel-dispatch, argument checking, iteration of +// ChunkedArray inputs and wrapping of outputs. +// +// To affect the execution options, you must call SetExecCtx and pass +// the resulting context in here. +func CallFunction(ctx context.Context, funcName string, opts FunctionOptions, args ...Datum) (Datum, error) { + ectx := GetExecCtx(ctx) + fn, ok := ectx.Registry.GetFunction(funcName) + if !ok { + return nil, fmt.Errorf("%w: function '%s' not found", arrow.ErrKey, funcName) + } + + return fn.Execute(ctx, opts, args...) +} diff --git a/go/arrow/compute/exec_internals_test.go b/go/arrow/compute/exec_internals_test.go new file mode 100644 index 0000000000000..93960dd16b2a8 --- /dev/null +++ b/go/arrow/compute/exec_internals_test.go @@ -0,0 +1,583 @@ +// 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 ( + "bytes" + "context" + "fmt" + "testing" + + "github.com/apache/arrow/go/v10/arrow" + "github.com/apache/arrow/go/v10/arrow/array" + "github.com/apache/arrow/go/v10/arrow/bitutil" + "github.com/apache/arrow/go/v10/arrow/compute/internal/exec" + "github.com/apache/arrow/go/v10/arrow/internal/testing/gen" + "github.com/apache/arrow/go/v10/arrow/memory" + "github.com/apache/arrow/go/v10/arrow/scalar" + "github.com/stretchr/testify/suite" +) + +type ComputeInternalsTestSuite struct { + suite.Suite + + mem *memory.CheckedAllocator + + execCtx ExecCtx + ctx *exec.KernelCtx + rng gen.RandomArrayGenerator +} + +func (c *ComputeInternalsTestSuite) SetupTest() { + c.mem = memory.NewCheckedAllocator(memory.DefaultAllocator) + c.rng = gen.NewRandomArrayGenerator(0, c.mem) + + c.resetCtx() +} + +func (c *ComputeInternalsTestSuite) TearDownTest() { + c.mem.AssertSize(c.T(), 0) +} + +func (c *ComputeInternalsTestSuite) assertArrayEqual(expected, got arrow.Array) { + c.Truef(array.Equal(expected, got), "expected: %s\ngot: %s", expected, got) +} + +func (c *ComputeInternalsTestSuite) assertDatumEqual(expected arrow.Array, got Datum) { + arr := got.(*ArrayDatum).MakeArray() + defer arr.Release() + c.Truef(array.Equal(expected, arr), "expected: %s\ngot: %s", expected, arr) +} + +func (c *ComputeInternalsTestSuite) resetCtx() { + c.execCtx = ExecCtx{Registry: GetFunctionRegistry(), + ChunkSize: DefaultMaxChunkSize, PreallocContiguous: true} + c.ctx = &exec.KernelCtx{Ctx: SetExecCtx(context.Background(), c.execCtx)} +} + +func (c *ComputeInternalsTestSuite) getBoolArr(sz int64, trueprob, nullprob float64) arrow.Array { + return c.rng.Boolean(sz, trueprob, nullprob) +} + +func (c *ComputeInternalsTestSuite) getUint8Arr(sz int64, nullprob float64) arrow.Array { + return c.rng.Uint8(sz, 0, 100, nullprob) +} + +func (c *ComputeInternalsTestSuite) getInt32Arr(sz int64, nullprob float64) arrow.Array { + return c.rng.Int32(sz, 0, 1000, nullprob) +} + +func (c *ComputeInternalsTestSuite) getFloat64Arr(sz int64, nullprob float64) arrow.Array { + return c.rng.Float64(sz, 0, 1000, nullprob) +} + +func (c *ComputeInternalsTestSuite) getInt32Chunked(szs []int64) *arrow.Chunked { + chunks := make([]arrow.Array, 0) + for i, s := range szs { + chunks = append(chunks, c.getInt32Arr(s, 0.1)) + defer chunks[i].Release() + } + return arrow.NewChunked(arrow.PrimitiveTypes.Int32, chunks) +} + +func (c *ComputeInternalsTestSuite) assertValidityZeroExtraBits(data []byte, length, offset int) { + bitExtent := ((offset + length + 7) / 8) * 8 + for i := offset + length; i < bitExtent; i++ { + c.False(bitutil.BitIsSet(data, i)) + } +} + +type PropagateNullsSuite struct { + ComputeInternalsTestSuite +} + +func (p *PropagateNullsSuite) TestUnknownNullCountWithNullsZeroCopies() { + const length int = 16 + bitmap := [8]byte{254, 0, 0, 0, 0, 0, 0, 0} + nulls := memory.NewBufferBytes(bitmap[:]) + + output := array.NewData(arrow.FixedWidthTypes.Boolean, length, []*memory.Buffer{nil, nil}, nil, 0, 0) + input := array.NewData(arrow.FixedWidthTypes.Boolean, length, []*memory.Buffer{nulls, nil}, nil, array.UnknownNullCount, 0) + + var outSpan exec.ArraySpan + outSpan.SetMembers(output) + batch := ExecBatch{Values: []Datum{NewDatum(input)}, Len: int64(length)} + p.NoError(propagateNulls(p.ctx, ExecSpanFromBatch(&batch), &outSpan)) + p.Same(nulls, outSpan.Buffers[0].Owner) + p.EqualValues(array.UnknownNullCount, outSpan.Nulls) + p.Equal(9, int(outSpan.Len)-bitutil.CountSetBits(outSpan.Buffers[0].Buf, int(outSpan.Offset), int(outSpan.Len))) +} + +func (p *PropagateNullsSuite) TestUnknownNullCountWithoutNulls() { + const length int = 16 + bitmap := [8]byte{255, 255, 0, 0, 0, 0, 0, 0} + nulls := memory.NewBufferBytes(bitmap[:]) + + output := array.NewData(arrow.FixedWidthTypes.Boolean, length, []*memory.Buffer{nil, nil}, nil, 0, 0) + input := array.NewData(arrow.FixedWidthTypes.Boolean, length, []*memory.Buffer{nulls, nil}, nil, array.UnknownNullCount, 0) + + var outSpan exec.ArraySpan + outSpan.SetMembers(output) + batch := ExecBatch{Values: []Datum{NewDatum(input)}, Len: int64(length)} + p.NoError(propagateNulls(p.ctx, ExecSpanFromBatch(&batch), &outSpan)) + p.EqualValues(-1, outSpan.Nulls) + p.Same(nulls, outSpan.Buffers[0].Owner) +} + +func (p *PropagateNullsSuite) TestSetAllNulls() { + const length int = 16 + checkSetAll := func(vals []Datum, prealloc bool) { + // fresh bitmap with all 1s + bitmapData := [2]byte{255, 255} + preallocatedMem := memory.NewBufferBytes(bitmapData[:]) + + output := &exec.ArraySpan{ + Type: arrow.FixedWidthTypes.Boolean, + Len: int64(length), + Nulls: array.UnknownNullCount, + } + + if prealloc { + output.Buffers[0].SetBuffer(preallocatedMem) + } + + batch := &ExecBatch{Values: vals, Len: int64(length)} + p.NoError(propagateNulls(p.ctx, ExecSpanFromBatch(batch), output)) + + if prealloc { + // ensure that the buffer object is the same when we pass preallocated + // memory to it + p.Same(preallocatedMem, output.Buffers[0].Owner) + } else { + defer output.Buffers[0].Owner.Release() + } + + p.NotNil(output.Buffers[0].Buf) + expected := [2]byte{0, 0} + p.True(bytes.Equal(expected[:], output.Buffers[0].Buf)) + } + + var vals []Datum + const trueProb float64 = 0.5 + p.Run("Null Scalar", func() { + i32Val := scalar.MakeScalar(int32(3)) + vals = []Datum{NewDatum(i32Val), NewDatum(scalar.MakeNullScalar(arrow.FixedWidthTypes.Boolean))} + checkSetAll(vals, true) + checkSetAll(vals, false) + + arr := p.getBoolArr(int64(length), trueProb, 0) + defer arr.Release() + vals[0] = NewDatum(arr) + defer vals[0].Release() + checkSetAll(vals, true) + checkSetAll(vals, false) + }) + + p.Run("one all null", func() { + arrAllNulls := p.getBoolArr(int64(length), trueProb, 1) + defer arrAllNulls.Release() + arrHalf := p.getBoolArr(int64(length), trueProb, 0.5) + defer arrHalf.Release() + vals = []Datum{NewDatum(arrHalf), NewDatum(arrAllNulls)} + defer vals[0].Release() + defer vals[1].Release() + + checkSetAll(vals, true) + checkSetAll(vals, false) + }) + + p.Run("one value is NullType", func() { + nullarr := array.NewNull(length) + arr := p.getBoolArr(int64(length), trueProb, 0) + defer arr.Release() + vals = []Datum{NewDatum(arr), NewDatum(nullarr)} + defer vals[0].Release() + checkSetAll(vals, true) + checkSetAll(vals, false) + }) + + p.Run("Other scenarios", func() { + // an all-null bitmap is zero-copied over, even though + // there is a null-scalar earlier in the batch + outSpan := &exec.ArraySpan{ + Type: arrow.FixedWidthTypes.Boolean, + Len: int64(length), + } + arrAllNulls := p.getBoolArr(int64(length), trueProb, 1) + defer arrAllNulls.Release() + + batch := &ExecBatch{ + Values: []Datum{ + NewDatum(scalar.MakeNullScalar(arrow.FixedWidthTypes.Boolean)), + NewDatum(arrAllNulls), + }, + Len: int64(length), + } + defer batch.Values[1].Release() + + p.NoError(propagateNulls(p.ctx, ExecSpanFromBatch(batch), outSpan)) + p.Same(arrAllNulls.Data().Buffers()[0], outSpan.Buffers[0].Owner) + outSpan.Buffers[0].Owner.Release() + }) +} + +func (p *PropagateNullsSuite) TestSingleValueWithNulls() { + const length int64 = 100 + arr := p.getBoolArr(length, 0.5, 0.5) + defer arr.Release() + + checkSliced := func(offset int64, prealloc bool, outOffset int64) { + // unaligned bitmap, zero copy not possible + sliced := array.NewSlice(arr, offset, int64(arr.Len())) + defer sliced.Release() + vals := []Datum{NewDatum(sliced)} + defer vals[0].Release() + + output := &exec.ArraySpan{ + Type: arrow.FixedWidthTypes.Boolean, + Len: vals[0].Len(), + Offset: outOffset, + } + + batch := &ExecBatch{Values: vals, Len: vals[0].Len()} + + var preallocatedBitmap *memory.Buffer + if prealloc { + preallocatedBitmap = memory.NewResizableBuffer(p.mem) + preallocatedBitmap.Resize(int(bitutil.BytesForBits(int64(sliced.Len()) + outOffset))) + defer preallocatedBitmap.Release() + output.Buffers[0].SetBuffer(preallocatedBitmap) + output.Buffers[0].SelfAlloc = true + } else { + p.EqualValues(0, output.Offset) + } + + p.NoError(propagateNulls(p.ctx, ExecSpanFromBatch(batch), output)) + if !prealloc { + parentBuf := arr.Data().Buffers()[0] + if offset == 0 { + // validity bitmap same, no slice + p.Same(parentBuf, output.Buffers[0].Owner) + } else if offset%8 == 0 { + // validity bitmap sliced + p.NotSame(parentBuf, output.Buffers[0].Owner) + p.Same(parentBuf, output.Buffers[0].Owner.Parent()) + defer output.Buffers[0].Owner.Release() + } else { + // new memory for offset not 0 mod 8 + p.NotSame(parentBuf, output.Buffers[0].Owner) + p.Nil(output.Buffers[0].Owner.Parent()) + defer output.Buffers[0].Owner.Release() + } + } else { + // preallocated, so check that the validity bitmap is unbothered + p.Same(preallocatedBitmap, output.Buffers[0].Owner) + } + + p.EqualValues(sliced.NullN(), output.UpdateNullCount()) + p.True(bitutil.BitmapEquals( + sliced.NullBitmapBytes(), output.Buffers[0].Buf, + int64(sliced.Data().Offset()), output.Offset, output.Len)) + p.assertValidityZeroExtraBits(output.Buffers[0].Buf, int(output.Len), int(output.Offset)) + } + + tests := []struct { + offset, outoffset int64 + prealloc bool + }{ + {8, 0, false}, + {7, 0, false}, + {8, 0, true}, + {7, 0, true}, + {8, 4, true}, + {7, 4, true}, + } + + for _, tt := range tests { + name := fmt.Sprintf("off=%d,prealloc=%t,outoff=%d", tt.offset, tt.prealloc, tt.outoffset) + p.Run(name, func() { + checkSliced(tt.offset, tt.prealloc, tt.outoffset) + }) + } +} + +func (p *PropagateNullsSuite) TestIntersectsNulls() { + const length = 16 + var ( + // 0b01111111 0b11001111 + bitmap1 = [8]byte{127, 207, 0, 0, 0, 0, 0, 0} + // 0b11111110 0b01111111 + bitmap2 = [8]byte{254, 127, 0, 0, 0, 0, 0, 0} + // 0b11101111 0b11111110 + bitmap3 = [8]byte{239, 254, 0, 0, 0, 0, 0, 0} + ) + + arr1 := array.NewData(arrow.FixedWidthTypes.Boolean, length, + []*memory.Buffer{memory.NewBufferBytes(bitmap1[:]), nil}, nil, array.UnknownNullCount, 0) + arr2 := array.NewData(arrow.FixedWidthTypes.Boolean, length, + []*memory.Buffer{memory.NewBufferBytes(bitmap2[:]), nil}, nil, array.UnknownNullCount, 0) + arr3 := array.NewData(arrow.FixedWidthTypes.Boolean, length, + []*memory.Buffer{memory.NewBufferBytes(bitmap3[:]), nil}, nil, array.UnknownNullCount, 0) + + checkCase := func(vals []Datum, exNullCount int, exBitmap []byte, prealloc bool, outoffset int) { + batch := &ExecBatch{Values: vals, Len: length} + + output := &exec.ArraySpan{Type: arrow.FixedWidthTypes.Boolean, Len: length} + + var nulls *memory.Buffer + if prealloc { + // make the buffer one byte bigger so we can have non-zero offsets + nulls = memory.NewResizableBuffer(p.mem) + nulls.Resize(3) + defer nulls.Release() + output.Buffers[0].SetBuffer(nulls) + output.Buffers[0].SelfAlloc = true + } else { + // non-zero output offset not permitted unless output memory is preallocated + p.Equal(0, outoffset) + } + + output.Offset = int64(outoffset) + + p.NoError(propagateNulls(p.ctx, ExecSpanFromBatch(batch), output)) + + // preallocated memory used + if prealloc { + p.Same(nulls, output.Buffers[0].Owner) + } else { + defer output.Buffers[0].Owner.Release() + } + + p.EqualValues(array.UnknownNullCount, output.Nulls) + p.EqualValues(exNullCount, output.UpdateNullCount()) + + p.True(bitutil.BitmapEquals(exBitmap, output.Buffers[0].Buf, 0, output.Offset, length)) + p.assertValidityZeroExtraBits(output.Buffers[0].Buf, int(output.Len), int(output.Offset)) + } + + p.Run("0b01101110 0b01001110", func() { + // 0b01101110 0b01001110 + expected := [2]byte{110, 78} + checkCase([]Datum{NewDatum(arr1), NewDatum(arr2), NewDatum(arr3)}, 7, expected[:], false, 0) + checkCase([]Datum{NewDatum(arr1), NewDatum(arr2), NewDatum(arr3)}, 7, expected[:], true, 0) + checkCase([]Datum{NewDatum(arr1), NewDatum(arr2), NewDatum(arr3)}, 7, expected[:], true, 4) + }) + + p.Run("0b01111110 0b01001111", func() { + expected := [2]byte{126, 79} + checkCase([]Datum{NewDatum(arr1), NewDatum(arr2)}, 5, expected[:], false, 0) + checkCase([]Datum{NewDatum(arr1), NewDatum(arr2)}, 5, expected[:], true, 4) + }) +} + +func TestComputeInternals(t *testing.T) { + suite.Run(t, new(PropagateNullsSuite)) +} + +type ExecSpanItrSuite struct { + ComputeInternalsTestSuite + + iter spanIterator +} + +func (e *ExecSpanItrSuite) setupIterator(batch *ExecBatch, maxChunk int64) { + var err error + _, e.iter, err = iterateExecSpans(batch, maxChunk, true) + e.NoError(err) +} + +func (e *ExecSpanItrSuite) checkIteration(input *ExecBatch, chunksize int, exBatchSizes []int) { + e.setupIterator(input, int64(chunksize)) + var ( + batch exec.ExecSpan + curPos int64 + pos int64 + next bool + ) + + for _, sz := range exBatchSizes { + batch, pos, next = e.iter() + e.True(next) + e.EqualValues(sz, batch.Len) + + for j, val := range input.Values { + switch val := val.(type) { + case *ScalarDatum: + e.Truef(scalar.Equals(batch.Values[j].Scalar, val.Value), "expected: %s\ngot: %s", val.Value, batch.Values[j].Scalar) + case *ArrayDatum: + arr := val.MakeArray() + sl := array.NewSlice(arr, curPos, curPos+batch.Len) + got := batch.Values[j].Array.MakeArray() + + e.Truef(array.Equal(sl, got), "expected: %s\ngot: %s", sl, got) + + got.Release() + arr.Release() + sl.Release() + case *ChunkedDatum: + carr := val.Value + if batch.Len == 0 { + e.Zero(carr.Len()) + } else { + chkd := array.NewChunkedSlice(carr, curPos, curPos+batch.Len) + defer chkd.Release() + e.Len(chkd.Chunks(), 1) + got := batch.Values[j].Array.MakeArray() + defer got.Release() + e.Truef(array.Equal(got, chkd.Chunk(0)), "expected: %s\ngot: %s", chkd.Chunk(0), got) + } + } + } + + curPos += int64(sz) + e.EqualValues(curPos, pos) + } + + batch, pos, next = e.iter() + e.Zero(batch) + e.False(next) + e.EqualValues(input.Len, pos) +} + +func (e *ExecSpanItrSuite) TestBasics() { + const length = 100 + + arr1 := e.getInt32Arr(length, 0.1) + defer arr1.Release() + arr2 := e.getFloat64Arr(length, 0.1) + defer arr2.Release() + + input := &ExecBatch{ + Len: length, + Values: []Datum{NewDatum(arr1), NewDatum(arr2), NewDatum(int32(3))}, + } + defer func() { + for _, v := range input.Values { + v.Release() + } + }() + + e.Run("simple", func() { + e.setupIterator(input, DefaultMaxChunkSize) + + batch, pos, next := e.iter() + e.True(next) + e.Len(batch.Values, 3) + e.EqualValues(length, batch.Len) + e.EqualValues(length, pos) + + in1 := input.Values[0].(*ArrayDatum).MakeArray() + defer in1.Release() + in2 := input.Values[1].(*ArrayDatum).MakeArray() + defer in2.Release() + out1 := batch.Values[0].Array.MakeArray() + defer out1.Release() + out2 := batch.Values[1].Array.MakeArray() + defer out2.Release() + + e.Truef(array.Equal(in1, out1), "expected: %s\ngot: %s", in1, out1) + e.Truef(array.Equal(in2, out2), "expected: %s\ngot: %s", in2, out2) + e.True(scalar.Equals(input.Values[2].(*ScalarDatum).Value, batch.Values[2].Scalar), input.Values[2].(*ScalarDatum).Value, batch.Values[2].Scalar) + + _, pos, next = e.iter() + e.EqualValues(length, pos) + e.False(next) + }) + + e.Run("iterations", func() { + e.checkIteration(input, 16, []int{16, 16, 16, 16, 16, 16, 4}) + }) +} + +func (e *ExecSpanItrSuite) TestInputValidation() { + arr1 := e.getInt32Arr(10, 0.1) + defer arr1.Release() + arr2 := e.getInt32Arr(9, 0.1) + defer arr2.Release() + + // length mismatch + batch := &ExecBatch{ + Values: []Datum{&ArrayDatum{arr1.Data()}, &ArrayDatum{arr2.Data()}}, + Len: 10, + } + + _, _, err := iterateExecSpans(batch, DefaultMaxChunkSize, true) + e.ErrorIs(err, arrow.ErrInvalid) + + // swap order of input + batch.Values = []Datum{&ArrayDatum{arr2.Data()}, &ArrayDatum{arr1.Data()}} + + _, _, err = iterateExecSpans(batch, DefaultMaxChunkSize, true) + e.ErrorIs(err, arrow.ErrInvalid) + + batch.Values = []Datum{&ArrayDatum{arr1.Data()}} + _, _, err = iterateExecSpans(batch, DefaultMaxChunkSize, true) + e.NoError(err) +} + +func (e *ExecSpanItrSuite) TestChunkedArrays() { + arr1 := e.getInt32Chunked([]int64{0, 20, 10}) + defer arr1.Release() + arr2 := e.getInt32Chunked([]int64{15, 15}) + defer arr2.Release() + arr3 := e.getInt32Arr(30, 0.1) + defer arr3.Release() + + batch := &ExecBatch{ + Values: []Datum{ + &ChunkedDatum{arr1}, &ChunkedDatum{arr2}, &ArrayDatum{arr3.Data()}, + NewDatum(int32(5)), NewDatum(scalar.MakeNullScalar(arrow.FixedWidthTypes.Boolean))}, + Len: 30, + } + + e.checkIteration(batch, 10, []int{10, 5, 5, 10}) + e.checkIteration(batch, 20, []int{15, 5, 10}) + e.checkIteration(batch, 30, []int{15, 5, 10}) +} + +func (e *ExecSpanItrSuite) TestZeroLengthInput() { + carr := arrow.NewChunked(arrow.PrimitiveTypes.Int32, []arrow.Array{}) + checkArgs := func(batch *ExecBatch) { + _, itr, err := iterateExecSpans(batch, DefaultMaxChunkSize, true) + e.NoError(err) + itrSpan, _, next := itr() + + e.False(next) + e.Zero(itrSpan) + } + + input := &ExecBatch{Len: 0} + + // zero-length chunkedarray with zero chunks + input.Values = []Datum{&ChunkedDatum{carr}} + checkArgs(input) + + // zero-length array + arr := e.getInt32Arr(0, 0.1) + defer arr.Release() + input.Values = []Datum{&ArrayDatum{arr.Data()}} + checkArgs(input) + + // chunkedarray with single empty chunk + carr = e.getInt32Chunked([]int64{0}) + input.Values = []Datum{&ChunkedDatum{carr}} + checkArgs(input) +} + +func TestExecSpanIterator(t *testing.T) { + suite.Run(t, new(ExecSpanItrSuite)) +} diff --git a/go/arrow/compute/exec_test.go b/go/arrow/compute/exec_test.go new file mode 100644 index 0000000000000..df0c67eeffdaf --- /dev/null +++ b/go/arrow/compute/exec_test.go @@ -0,0 +1,377 @@ +// 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 ( + "strings" + "testing" + + "github.com/apache/arrow/go/v10/arrow" + "github.com/apache/arrow/go/v10/arrow/array" + "github.com/apache/arrow/go/v10/arrow/bitutil" + "github.com/apache/arrow/go/v10/arrow/compute/internal/exec" + "github.com/apache/arrow/go/v10/arrow/internal/debug" + "github.com/apache/arrow/go/v10/arrow/scalar" + "github.com/stretchr/testify/suite" +) + +func ExecCopyArray(ctx *exec.KernelCtx, batch *exec.ExecSpan, out *exec.ExecResult) error { + debug.Assert(len(batch.Values) == 1, "wrong number of values") + valueSize := int64(batch.Values[0].Type().(arrow.FixedWidthDataType).BitWidth() / 8) + + arg0 := batch.Values[0].Array + dst := out.Buffers[1].Buf[out.Offset*valueSize:] + src := arg0.Buffers[1].Buf[arg0.Offset*valueSize:] + copy(dst, src[:batch.Len*valueSize]) + return nil +} + +func ExecComputedBitmap(ctx *exec.KernelCtx, batch *exec.ExecSpan, out *exec.ExecResult) error { + // propagate nulls not used. check that out bitmap isn't the same already + // as the input bitmap + arg0 := batch.Values[0].Array + if bitutil.CountSetBits(arg0.Buffers[1].Buf, int(arg0.Offset), int(batch.Len)) > 0 { + // check that the bitmap hasn't already been copied + debug.Assert(!bitutil.BitmapEquals(arg0.Buffers[0].Buf, out.Buffers[0].Buf, + arg0.Offset, out.Offset, batch.Len), "bitmap should not have already been copied") + } + + bitutil.CopyBitmap(arg0.Buffers[0].Buf, int(arg0.Offset), int(batch.Len), out.Buffers[0].Buf, int(out.Offset)) + return ExecCopyArray(ctx, batch, out) +} + +func ExecNoPreallocatedData(ctx *exec.KernelCtx, batch *exec.ExecSpan, out *exec.ExecResult) error { + // validity preallocated, not data + debug.Assert(out.Offset == 0, "invalid offset for non-prealloc") + valueSize := int64(batch.Values[0].Type().(arrow.FixedWidthDataType).BitWidth() / 8) + out.Buffers[1].SetBuffer(ctx.Allocate(int(out.Len * valueSize))) + out.Buffers[1].SelfAlloc = true + return ExecCopyArray(ctx, batch, out) +} + +func ExecNoPreallocatedAnything(ctx *exec.KernelCtx, batch *exec.ExecSpan, out *exec.ExecResult) error { + // neither validity nor data preallocated + debug.Assert(out.Offset == 0, "invalid offset for non-prealloc") + out.Buffers[0].SetBuffer(ctx.AllocateBitmap(out.Len)) + out.Buffers[0].SelfAlloc = true + arg0 := batch.Values[0].Array + bitutil.CopyBitmap(arg0.Buffers[0].Buf, int(arg0.Offset), int(batch.Len), out.Buffers[0].Buf, 0) + + // reuse kernel that allocates data + return ExecNoPreallocatedData(ctx, batch, out) +} + +type ExampleOptions struct { + Value scalar.Scalar +} + +func (e *ExampleOptions) TypeName() string { return "example" } + +type ExampleState struct { + Value scalar.Scalar +} + +func InitStateful(_ *exec.KernelCtx, args exec.KernelInitArgs) (exec.KernelState, error) { + value := args.Options.(*ExampleOptions).Value + return &ExampleState{Value: value}, nil +} + +func ExecStateful(ctx *exec.KernelCtx, batch *exec.ExecSpan, out *exec.ExecResult) error { + state := ctx.State.(*ExampleState) + multiplier := state.Value.(*scalar.Int32).Value + + arg0 := batch.Values[0].Array + arg0Data := exec.GetSpanValues[int32](&arg0, 1) + dst := exec.GetSpanValues[int32](out, 1) + for i, v := range arg0Data { + dst[i] = v * multiplier + } + return nil +} + +func ExecAddInt32(ctx *exec.KernelCtx, batch *exec.ExecSpan, out *exec.ExecResult) error { + left := exec.GetSpanValues[int32](&batch.Values[0].Array, 1) + right := exec.GetSpanValues[int32](&batch.Values[1].Array, 1) + outValues := exec.GetSpanValues[int32](out, 1) + for i := 0; i < int(batch.Len); i++ { + outValues[i] = left[i] + right[i] + } + return nil +} + +type CallScalarFuncSuite struct { + ComputeInternalsTestSuite +} + +func (c *CallScalarFuncSuite) addCopyFuncs() { + registry = GetFunctionRegistry() + + fn := NewScalarFunction("test_copy", Unary(), EmptyFuncDoc) + types := []arrow.DataType{arrow.PrimitiveTypes.Uint8, arrow.PrimitiveTypes.Int32, arrow.PrimitiveTypes.Float64} + for _, t := range types { + c.NoError(fn.AddNewKernel([]exec.InputType{exec.NewExactInput(t)}, + exec.NewOutputType(t), ExecCopyArray, nil)) + } + c.True(registry.AddFunction(fn, false)) + + // a version which doesn't want the executor to call propagatenulls + fn2 := NewScalarFunction("test_copy_computed_bitmap", Unary(), EmptyFuncDoc) + kernel := exec.NewScalarKernel([]exec.InputType{exec.NewExactInput(arrow.PrimitiveTypes.Uint8)}, + exec.NewOutputType(arrow.PrimitiveTypes.Uint8), ExecComputedBitmap, nil) + kernel.NullHandling = exec.NullComputedPrealloc + c.NoError(fn2.AddKernel(kernel)) + c.True(registry.AddFunction(fn2, false)) +} + +func (c *CallScalarFuncSuite) addNoPreallocFuncs() { + registry = GetFunctionRegistry() + + // a function that allocates its own output memory. we have cases + // for both non-preallocated data and non-preallocated bitmap + f1 := NewScalarFunction("test_nopre_data", Unary(), EmptyFuncDoc) + f2 := NewScalarFunction("test_nopre_validity_or_data", Unary(), EmptyFuncDoc) + + kernel := exec.NewScalarKernel( + []exec.InputType{exec.NewExactInput(arrow.PrimitiveTypes.Uint8)}, + exec.NewOutputType(arrow.PrimitiveTypes.Uint8), + ExecNoPreallocatedData, nil) + kernel.MemAlloc = exec.MemNoPrealloc + c.NoError(f1.AddKernel(kernel)) + + kernel.ExecFn = ExecNoPreallocatedAnything + kernel.NullHandling = exec.NullComputedNoPrealloc + c.NoError(f2.AddKernel(kernel)) + + c.True(registry.AddFunction(f1, false)) + c.True(registry.AddFunction(f2, false)) +} + +func (c *CallScalarFuncSuite) addStatefulFunc() { + registry := GetFunctionRegistry() + + // this functions behavior depends on a static parameter that + // is made available to the execution through its options object + fn := NewScalarFunction("test_stateful", Unary(), EmptyFuncDoc) + + c.NoError(fn.AddNewKernel([]exec.InputType{exec.NewExactInput(arrow.PrimitiveTypes.Int32)}, + exec.NewOutputType(arrow.PrimitiveTypes.Int32), ExecStateful, InitStateful)) + + c.True(registry.AddFunction(fn, false)) +} + +func (c *CallScalarFuncSuite) addScalarFunc() { + registry := GetFunctionRegistry() + + fn := NewScalarFunction("test_scalar_add_int32", Binary(), EmptyFuncDoc) + c.NoError(fn.AddNewKernel([]exec.InputType{ + exec.NewExactInput(arrow.PrimitiveTypes.Int32), + exec.NewExactInput(arrow.PrimitiveTypes.Int32)}, + exec.NewOutputType(arrow.PrimitiveTypes.Int32), ExecAddInt32, nil)) + c.True(registry.AddFunction(fn, false)) +} + +func (c *CallScalarFuncSuite) SetupSuite() { + c.addCopyFuncs() + c.addNoPreallocFuncs() + c.addStatefulFunc() + c.addScalarFunc() +} + +func (c *CallScalarFuncSuite) TestArgumentValidation() { + // copy accepts only a single array arg + arr := c.getInt32Arr(10, 0.1) + defer arr.Release() + d1 := &ArrayDatum{Value: arr.Data()} + + c.Run("too many args", func() { + args := []Datum{d1, d1} + _, err := CallFunction(c.ctx.Ctx, "test_copy", nil, args...) + c.ErrorIs(err, arrow.ErrInvalid) + }) + + c.Run("too few args", func() { + _, err := CallFunction(c.ctx.Ctx, "test_copy", nil) + c.ErrorIs(err, arrow.ErrInvalid) + }) + + d1Scalar := NewDatum(int32(5)) + result, err := CallFunction(c.ctx.Ctx, "test_copy", nil, d1) + c.NoError(err) + result.Release() + result, err = CallFunction(c.ctx.Ctx, "test_copy", nil, d1Scalar) + c.NoError(err) + result.Release() +} + +func (c *CallScalarFuncSuite) TestPreallocationCases() { + nullProb := float64(0.2) + arr := c.getUint8Arr(100, nullProb) + defer arr.Release() + + funcNames := []string{"test_copy", "test_copy_computed_bitmap"} + for _, funcName := range funcNames { + c.Run(funcName, func() { + c.resetCtx() + + c.Run("single output default", func() { + result, err := CallFunction(c.ctx.Ctx, funcName, nil, &ArrayDatum{arr.Data()}) + c.NoError(err) + defer result.Release() + c.Equal(KindArray, result.Kind()) + c.assertDatumEqual(arr, result) + }) + + c.Run("exec chunks", func() { + // set the exec_chunksize to be smaller so now we have + // several invocations of the kernel, + // but still only one output array + c.execCtx.ChunkSize = 80 + result, err := CallFunction(SetExecCtx(c.ctx.Ctx, c.execCtx), funcName, nil, &ArrayDatum{arr.Data()}) + c.NoError(err) + defer result.Release() + c.Equal(KindArray, result.Kind()) + c.assertDatumEqual(arr, result) + }) + + c.Run("not multiple 8 chunk", func() { + // chunksize is not a multiple of 8 + c.execCtx.ChunkSize = 11 + result, err := CallFunction(SetExecCtx(c.ctx.Ctx, c.execCtx), funcName, nil, &ArrayDatum{arr.Data()}) + c.NoError(err) + defer result.Release() + c.Equal(KindArray, result.Kind()) + c.assertDatumEqual(arr, result) + }) + + c.Run("chunked", func() { + // input is chunked, output is one big chunk + chk1, chk2 := array.NewSlice(arr, 0, 10), array.NewSlice(arr, 10, int64(arr.Len())) + defer chk1.Release() + defer chk2.Release() + carr := arrow.NewChunked(arr.DataType(), []arrow.Array{chk1, chk2}) + defer carr.Release() + + result, err := CallFunction(SetExecCtx(c.ctx.Ctx, c.execCtx), funcName, nil, &ChunkedDatum{carr}) + c.NoError(err) + defer result.Release() + c.Equal(KindChunked, result.Kind()) + actual := result.(*ChunkedDatum).Value + c.Len(actual.Chunks(), 1) + c.Truef(array.ChunkedEqual(actual, carr), "expected: %s\ngot: %s", carr, actual) + }) + + c.Run("independent", func() { + // preallocate independently for each batch + c.execCtx.PreallocContiguous = false + c.execCtx.ChunkSize = 40 + result, err := CallFunction(SetExecCtx(c.ctx.Ctx, c.execCtx), funcName, nil, &ArrayDatum{arr.Data()}) + c.NoError(err) + defer result.Release() + c.Equal(KindChunked, result.Kind()) + + carr := result.(*ChunkedDatum).Value + c.Len(carr.Chunks(), 3) + sl := array.NewSlice(arr, 0, 40) + defer sl.Release() + c.assertArrayEqual(sl, carr.Chunk(0)) + sl = array.NewSlice(arr, 40, 80) + defer sl.Release() + c.assertArrayEqual(sl, carr.Chunk(1)) + sl = array.NewSlice(arr, 80, int64(arr.Len())) + defer sl.Release() + c.assertArrayEqual(sl, carr.Chunk(2)) + }) + }) + } +} + +func (c *CallScalarFuncSuite) TestBasicNonStandardCases() { + // test some more cases + // + // * validity bitmap computed by kernel rather than propagate nulls + // * data not pre-allocated + // * validity bitmap not pre-allocated + + nullProb := float64(0.2) + arr := c.getUint8Arr(1000, nullProb) + defer arr.Release() + args := []Datum{&ArrayDatum{arr.Data()}} + + for _, funcName := range []string{"test_nopre_data", "test_nopre_validity_or_data"} { + c.Run("funcName", func() { + c.resetCtx() + c.Run("single output default", func() { + result, err := CallFunction(c.ctx.Ctx, funcName, nil, args...) + c.NoError(err) + defer result.Release() + c.Equal(KindArray, result.Kind()) + c.assertDatumEqual(arr, result) + }) + + c.Run("split into 3 chunks", func() { + c.execCtx.ChunkSize = 400 + result, err := CallFunction(SetExecCtx(c.ctx.Ctx, c.execCtx), funcName, nil, args...) + c.NoError(err) + defer result.Release() + + c.Equal(KindChunked, result.Kind()) + + carr := result.(*ChunkedDatum).Value + c.Len(carr.Chunks(), 3) + sl := array.NewSlice(arr, 0, 400) + defer sl.Release() + c.assertArrayEqual(sl, carr.Chunk(0)) + sl = array.NewSlice(arr, 400, 800) + defer sl.Release() + c.assertArrayEqual(sl, carr.Chunk(1)) + sl = array.NewSlice(arr, 800, int64(arr.Len())) + defer sl.Release() + c.assertArrayEqual(sl, carr.Chunk(2)) + }) + }) + } +} + +func (c *CallScalarFuncSuite) TestStatefulKernel() { + input, _, _ := array.FromJSON(c.mem, arrow.PrimitiveTypes.Int32, strings.NewReader(`[1, 2, 3, null, 5]`)) + defer input.Release() + + multiplier := scalar.MakeScalar(int32(2)) + expected, _, _ := array.FromJSON(c.mem, arrow.PrimitiveTypes.Int32, strings.NewReader(`[2, 4, 6, null, 10]`)) + defer expected.Release() + + options := &ExampleOptions{multiplier} + result, err := CallFunction(c.ctx.Ctx, "test_stateful", options, &ArrayDatum{input.Data()}) + c.NoError(err) + defer result.Release() + c.assertDatumEqual(expected, result) +} + +func (c *CallScalarFuncSuite) TestScalarFunction() { + args := []Datum{NewDatum(int32(5)), NewDatum(int32(7))} + result, err := CallFunction(c.ctx.Ctx, "test_scalar_add_int32", nil, args...) + c.NoError(err) + defer result.Release() + + c.Equal(KindScalar, result.Kind()) + expected := scalar.MakeScalar(int32(12)) + c.True(scalar.Equals(expected, result.(*ScalarDatum).Value)) +} + +func TestCallScalarFunctions(t *testing.T) { + suite.Run(t, new(CallScalarFuncSuite)) +} diff --git a/go/arrow/compute/executor.go b/go/arrow/compute/executor.go new file mode 100644 index 0000000000000..72f6cf4623b7c --- /dev/null +++ b/go/arrow/compute/executor.go @@ -0,0 +1,802 @@ +// 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" + "math" + "sync" + + "github.com/apache/arrow/go/v10/arrow" + "github.com/apache/arrow/go/v10/arrow/array" + "github.com/apache/arrow/go/v10/arrow/bitutil" + "github.com/apache/arrow/go/v10/arrow/compute/internal/exec" + "github.com/apache/arrow/go/v10/arrow/internal" + "github.com/apache/arrow/go/v10/arrow/internal/debug" + "github.com/apache/arrow/go/v10/arrow/memory" + "github.com/apache/arrow/go/v10/arrow/scalar" +) + +// ExecCtx holds simple contextual information for execution +// such as the default ChunkSize for batch iteration, whether or not +// to ensure contiguous preallocations for kernels that want preallocation, +// and a reference to the desired function registry to use. +// +// An ExecCtx should be placed into a context.Context by using +// SetExecCtx and GetExecCtx to pass it along for execution. +type ExecCtx struct { + ChunkSize int64 + PreallocContiguous bool + Registry FunctionRegistry + ExecChannelSize int +} + +type ctxExecKey struct{} + +const DefaultMaxChunkSize = math.MaxInt64 + +// global default ExecCtx object, initialized with the +// default max chunk size, contiguous preallocations, and +// the default function registry. +var defaultExecCtx ExecCtx + +func init() { + defaultExecCtx.ChunkSize = DefaultMaxChunkSize + defaultExecCtx.PreallocContiguous = true + defaultExecCtx.Registry = GetFunctionRegistry() + defaultExecCtx.ExecChannelSize = 10 +} + +// SetExecCtx returns a new child context containing the passed in ExecCtx +func SetExecCtx(ctx context.Context, e ExecCtx) context.Context { + return context.WithValue(ctx, ctxExecKey{}, e) +} + +// GetExecCtx returns an embedded ExecCtx from the provided context. +// If it does not contain an ExecCtx, then the default one is returned. +func GetExecCtx(ctx context.Context) ExecCtx { + e, ok := ctx.Value(ctxExecKey{}).(ExecCtx) + if ok { + return e + } + return defaultExecCtx +} + +// ExecBatch is a unit of work for kernel execution. It contains a collection +// of Array and Scalar values. +// +// ExecBatch is semantically similar to a RecordBatch but for a SQL-style +// execution context. It represents a collection or records, but constant +// "columns" are represented by Scalar values rather than having to be +// converted into arrays with repeated values. +type ExecBatch struct { + Values []Datum + // Guarantee is a predicate Expression guaranteed to evaluate to true for + // all rows in this batch. + Guarantee Expression + // Len is the semantic length of this ExecBatch. When the values are + // all scalars, the length should be set to 1 for non-aggregate kernels. + // Otherwise the length is taken from the array values. Aggregate kernels + // can have an ExecBatch formed by projecting just the partition columns + // from a batch in which case it would have scalar rows with length > 1 + // + // If the array values are of length 0, then the length is 0 regardless of + // whether any values are Scalar. + Len int64 +} + +func (e ExecBatch) NumValues() int { return len(e.Values) } + +// simple struct for defining how to preallocate a particular buffer. +type bufferPrealloc struct { + bitWidth int + addLen int +} + +func allocateDataBuffer(ctx *exec.KernelCtx, length, bitWidth int) *memory.Buffer { + switch bitWidth { + case 1: + return ctx.AllocateBitmap(int64(length)) + default: + bufsiz := int(bitutil.BytesForBits(int64(length * bitWidth))) + return ctx.Allocate(bufsiz) + } +} + +func addComputeDataPrealloc(dt arrow.DataType, widths []bufferPrealloc) []bufferPrealloc { + if typ, ok := dt.(arrow.FixedWidthDataType); ok { + return append(widths, bufferPrealloc{bitWidth: typ.BitWidth()}) + } + + switch dt.ID() { + case arrow.BINARY, arrow.STRING, arrow.LIST, arrow.MAP: + return append(widths, bufferPrealloc{bitWidth: 32, addLen: 1}) + case arrow.LARGE_BINARY, arrow.LARGE_STRING, arrow.LARGE_LIST: + return append(widths, bufferPrealloc{bitWidth: 64, addLen: 1}) + } + return widths +} + +// enum to define a generalized assumption of the nulls in the inputs +type nullGeneralization int8 + +const ( + nullGenPerhapsNull nullGeneralization = iota + nullGenAllValid + nullGenAllNull +) + +func getNullGen(val *exec.ExecValue) nullGeneralization { + dtID := val.Type().ID() + switch { + case dtID == arrow.NULL: + return nullGenAllNull + case !internal.DefaultHasValidityBitmap(dtID): + return nullGenAllValid + case val.IsScalar(): + if val.Scalar.IsValid() { + return nullGenAllValid + } + return nullGenAllNull + default: + arr := val.Array + // do not count if they haven't been counted already + if arr.Nulls == 0 || arr.Buffers[0].Buf == nil { + return nullGenAllValid + } + + if arr.Nulls == arr.Len { + return nullGenAllNull + } + } + return nullGenPerhapsNull +} + +func getNullGenDatum(datum Datum) nullGeneralization { + var val exec.ExecValue + switch datum.Kind() { + case KindArray: + val.Array.SetMembers(datum.(*ArrayDatum).Value) + case KindScalar: + val.Scalar = datum.(*ScalarDatum).Value + case KindChunked: + return nullGenPerhapsNull + default: + debug.Assert(false, "should be array, scalar, or chunked!") + return nullGenPerhapsNull + } + return getNullGen(&val) +} + +// populate the validity bitmaps with the intersection of the nullity +// of the arguments. If a preallocated bitmap is not provided, then one +// will be allocated if needed (in some cases a bitmap can be zero-copied +// from the arguments). If any Scalar value is null, then the entire +// validity bitmap will be set to null. +func propagateNulls(ctx *exec.KernelCtx, batch *exec.ExecSpan, out *exec.ArraySpan) (err error) { + if out.Type.ID() == arrow.NULL { + // null output type is a no-op (rare but it happens) + return + } + + // this function is ONLY able to write into output with non-zero offset + // when the bitmap is preallocated. + if out.Offset != 0 && out.Buffers[0].Buf == nil { + return fmt.Errorf("%w: can only propagate nulls into pre-allocated memory when output offset is non-zero", arrow.ErrInvalid) + } + + var ( + arrsWithNulls = make([]*exec.ArraySpan, 0) + isAllNull bool + prealloc bool = out.Buffers[0].Buf != nil + ) + + for i := range batch.Values { + v := &batch.Values[i] + nullGen := getNullGen(v) + if nullGen == nullGenAllNull { + isAllNull = true + } + if nullGen != nullGenAllValid && v.IsArray() { + arrsWithNulls = append(arrsWithNulls, &v.Array) + } + } + + outBitmap := out.Buffers[0].Buf + if isAllNull { + // an all-null value gives us a short circuit opportunity + // output should all be null + out.Nulls = out.Len + if prealloc { + bitutil.SetBitsTo(outBitmap, out.Offset, out.Len, false) + return + } + + // walk all the values with nulls instead of breaking on the first + // in case we find a bitmap that can be reused in the non-preallocated case + for _, arr := range arrsWithNulls { + if arr.Nulls == arr.Len && arr.Buffers[0].Owner != nil { + buf := arr.GetBuffer(0) + buf.Retain() + out.Buffers[0].Buf = buf.Bytes() + out.Buffers[0].Owner = buf + return + } + } + + buf := ctx.AllocateBitmap(int64(out.Len)) + out.Buffers[0].Owner = buf + out.Buffers[0].Buf = buf.Bytes() + out.Buffers[0].SelfAlloc = true + bitutil.SetBitsTo(out.Buffers[0].Buf, out.Offset, out.Len, false) + return + } + + out.Nulls = array.UnknownNullCount + switch len(arrsWithNulls) { + case 0: + out.Nulls = 0 + if prealloc { + bitutil.SetBitsTo(outBitmap, out.Offset, out.Len, true) + } + case 1: + arr := arrsWithNulls[0] + out.Nulls = arr.Nulls + if prealloc { + bitutil.CopyBitmap(arr.Buffers[0].Buf, int(arr.Offset), int(arr.Len), outBitmap, int(out.Offset)) + return + } + + switch { + case arr.Offset == 0: + out.Buffers[0] = arr.Buffers[0] + out.Buffers[0].Owner.Retain() + case arr.Offset%8 == 0: + buf := memory.SliceBuffer(arr.GetBuffer(0), int(arr.Offset)/8, int(bitutil.BytesForBits(arr.Len))) + out.Buffers[0].Buf = buf.Bytes() + out.Buffers[0].Owner = buf + default: + buf := ctx.AllocateBitmap(int64(out.Len)) + out.Buffers[0].Owner = buf + out.Buffers[0].Buf = buf.Bytes() + out.Buffers[0].SelfAlloc = true + bitutil.CopyBitmap(arr.Buffers[0].Buf, int(arr.Offset), int(arr.Len), out.Buffers[0].Buf, 0) + } + return + + default: + if !prealloc { + buf := ctx.AllocateBitmap(int64(out.Len)) + out.Buffers[0].Owner = buf + out.Buffers[0].Buf = buf.Bytes() + out.Buffers[0].SelfAlloc = true + outBitmap = out.Buffers[0].Buf + } + + acc := func(left, right *exec.ArraySpan) { + debug.Assert(left.Buffers[0].Buf != nil, "invalid intersection for null propagation") + debug.Assert(right.Buffers[0].Buf != nil, "invalid intersection for null propagation") + bitutil.BitmapAnd(left.Buffers[0].Buf, right.Buffers[0].Buf, left.Offset, right.Offset, outBitmap, out.Offset, out.Len) + } + + acc(arrsWithNulls[0], arrsWithNulls[1]) + for _, arr := range arrsWithNulls[2:] { + acc(out, arr) + } + } + return +} + +func inferBatchLength(values []Datum) (length int64, allSame bool) { + length, allSame = -1, true + areAllScalar := true + for _, arg := range values { + switch arg := arg.(type) { + case *ArrayDatum: + argLength := arg.Len() + if length < 0 { + length = argLength + } else { + if length != argLength { + allSame = false + return + } + } + areAllScalar = false + case *ChunkedDatum: + argLength := arg.Len() + if length < 0 { + length = argLength + } else { + if length != argLength { + allSame = false + return + } + } + areAllScalar = false + } + } + + if areAllScalar && len(values) > 0 { + length = 1 + } else if length < 0 { + length = 0 + } + allSame = true + return +} + +// kernelExecutor is the interface for all executors to initialize and +// call kernel execution functions on batches. +type kernelExecutor interface { + // Init must be called *after* the kernel's init method and any + // KernelState must be set into the KernelCtx *before* calling + // this Init method. This is to faciliate the case where + // Init may be expensive and does not need to be called + // again for each execution of the kernel. For example, + // the same lookup table can be re-used for all scanned batches + // in a dataset filter. + Init(*exec.KernelCtx, exec.KernelInitArgs) error + // Execute the kernel for the provided batch and pass the resulting + // Datum values to the provided channel. + Execute(context.Context, *ExecBatch, chan<- Datum) error + // WrapResults exists for the case where an executor wants to post process + // the batches of result datums. Such as creating a ChunkedArray from + // multiple output batches or so on. Results from individual batch + // executions should be read from the out channel, and WrapResults should + // return the final Datum result. + WrapResults(ctx context.Context, out <-chan Datum, chunkedArgs bool) Datum + // CheckResultType checks the actual result type against the resolved + // output type. If the types don't match an error is returned + CheckResultType(out Datum) error + + clear() +} + +// the base implementation for executing non-aggregate kernels. +type nonAggExecImpl struct { + ctx *exec.KernelCtx + ectx ExecCtx + kernel exec.NonAggKernel + outType arrow.DataType + numOutBuf int + dataPrealloc []bufferPrealloc + preallocValidity bool +} + +func (e *nonAggExecImpl) clear() { + e.ctx, e.kernel, e.outType = nil, nil, nil + if e.dataPrealloc != nil { + e.dataPrealloc = e.dataPrealloc[:0] + } +} + +func (e *nonAggExecImpl) Init(ctx *exec.KernelCtx, args exec.KernelInitArgs) (err error) { + e.ctx, e.kernel = ctx, args.Kernel.(exec.NonAggKernel) + e.outType, err = e.kernel.GetSig().OutType.Resolve(ctx, args.Inputs) + e.ectx = GetExecCtx(ctx.Ctx) + return +} + +func (e *nonAggExecImpl) prepareOutput(length int) *exec.ExecResult { + var nullCount int = array.UnknownNullCount + + if e.kernel.GetNullHandling() == exec.NullNoOutput { + nullCount = 0 + } + + output := &exec.ArraySpan{ + Type: e.outType, + Len: int64(length), + Nulls: int64(nullCount), + } + + if e.preallocValidity { + buf := e.ctx.AllocateBitmap(int64(length)) + output.Buffers[0].Owner = buf + output.Buffers[0].Buf = buf.Bytes() + output.Buffers[0].SelfAlloc = true + } + + for i, pre := range e.dataPrealloc { + if pre.bitWidth >= 0 { + buf := allocateDataBuffer(e.ctx, length+pre.addLen, pre.bitWidth) + output.Buffers[i+1].Owner = buf + output.Buffers[i+1].Buf = buf.Bytes() + output.Buffers[i+1].SelfAlloc = true + } + } + + return output +} + +func (e *nonAggExecImpl) CheckResultType(out Datum) error { + typ := out.(ArrayLikeDatum).Type() + if typ != nil && !arrow.TypeEqual(e.outType, typ) { + return fmt.Errorf("%w: kernel type result mismatch: declared as %s, actual is %s", + arrow.ErrType, e.outType, typ) + } + return nil +} + +type spanIterator func() (exec.ExecSpan, int64, bool) + +type scalarExecutor struct { + nonAggExecImpl + + elideValidityBitmap bool + preallocAllBufs bool + preallocContiguous bool + allScalars bool + iter spanIterator + iterLen int64 +} + +func (s *scalarExecutor) Execute(ctx context.Context, batch *ExecBatch, data chan<- Datum) (err error) { + s.allScalars, s.iter, err = iterateExecSpans(batch, s.ectx.ChunkSize, true) + if err != nil { + return + } + + s.iterLen = batch.Len + + if batch.Len == 0 { + result := array.MakeArrayOfNull(exec.GetAllocator(s.ctx.Ctx), s.outType, 0) + defer result.Release() + out := &exec.ArraySpan{} + out.SetMembers(result.Data()) + return s.emitResult(out, data) + } + + if err = s.setupPrealloc(batch.Len, batch.Values); err != nil { + return + } + + return s.executeSpans(data) +} + +func (s *scalarExecutor) WrapResults(ctx context.Context, out <-chan Datum, hasChunked bool) Datum { + var ( + output Datum + acc []arrow.Array + ) + + toChunked := func() { + acc = output.(ArrayLikeDatum).Chunks() + output.Release() + output = nil + } + + // get first output + select { + case <-ctx.Done(): + return nil + case output = <-out: + // if the inputs contained at least one chunked array + // then we want to return chunked output + if hasChunked { + toChunked() + } + } + + for { + select { + case <-ctx.Done(): + // context is done, either cancelled or a timeout. + // either way, we end early and return what we've got so far. + return output + case o, ok := <-out: + if !ok { // channel closed, wrap it up + if output != nil { + return output + } + + for _, c := range acc { + defer c.Release() + } + + chkd := arrow.NewChunked(s.outType, acc) + defer chkd.Release() + return NewDatum(chkd) + } + + // if we get multiple batches of output, then we need + // to return it as a chunked array. + if acc == nil { + toChunked() + } + + defer o.Release() + if o.Len() == 0 { // skip any empty batches + continue + } + + acc = append(acc, o.(*ArrayDatum).MakeArray()) + } + } +} + +func (s *scalarExecutor) executeSpans(data chan<- Datum) (err error) { + var ( + input exec.ExecSpan + output exec.ExecResult + next bool + ) + + if s.preallocContiguous { + // make one big output alloc + prealloc := s.prepareOutput(int(s.iterLen)) + output = *prealloc + + output.Offset = 0 + var resultOffset int64 + var nextOffset int64 + for err == nil { + if input, nextOffset, next = s.iter(); !next { + break + } + output.SetSlice(resultOffset, input.Len) + err = s.executeSingleSpan(&input, &output) + resultOffset = nextOffset + } + if err != nil { + return + } + + return s.emitResult(prealloc, data) + } + + // fully preallocating, but not contiguously + // we (maybe) preallocate only for the output of processing + // the current chunk + for err == nil { + if input, _, next = s.iter(); !next { + break + } + + output = *s.prepareOutput(int(input.Len)) + if err = s.executeSingleSpan(&input, &output); err != nil { + return + } + err = s.emitResult(&output, data) + } + + return +} + +func (s *scalarExecutor) executeSingleSpan(input *exec.ExecSpan, out *exec.ExecResult) error { + switch { + case out.Type.ID() == arrow.NULL: + out.Nulls = out.Len + case s.kernel.GetNullHandling() == exec.NullIntersection: + if !s.elideValidityBitmap { + propagateNulls(s.ctx, input, out) + } + case s.kernel.GetNullHandling() == exec.NullNoOutput: + out.Nulls = 0 + } + return s.kernel.Exec(s.ctx, input, out) +} + +func (s *scalarExecutor) setupPrealloc(totalLen int64, args []Datum) error { + s.numOutBuf = len(s.outType.Layout().Buffers) + outTypeID := s.outType.ID() + // default to no validity pre-allocation for the following cases: + // - Output Array is NullArray + // - kernel.NullHandling is ComputeNoPrealloc or OutputNotNull + s.preallocValidity = false + + if outTypeID != arrow.NULL { + switch s.kernel.GetNullHandling() { + case exec.NullComputedPrealloc: + s.preallocValidity = true + case exec.NullIntersection: + s.elideValidityBitmap = true + for _, a := range args { + nullGen := getNullGenDatum(a) == nullGenAllValid + s.elideValidityBitmap = s.elideValidityBitmap && nullGen + } + s.preallocValidity = !s.elideValidityBitmap + case exec.NullNoOutput: + s.elideValidityBitmap = true + } + } + + if s.kernel.GetMemAlloc() == exec.MemPrealloc { + s.dataPrealloc = addComputeDataPrealloc(s.outType, s.dataPrealloc) + } + + // validity bitmap either preallocated or elided, and all data buffers allocated + // this is basically only true for primitive types that are not dict-encoded + s.preallocAllBufs = + ((s.preallocValidity || s.elideValidityBitmap) && len(s.dataPrealloc) == (s.numOutBuf-1) && + !arrow.IsNested(outTypeID) && outTypeID != arrow.DICTIONARY) + + // contiguous prealloc only possible on non-nested types if all + // buffers are preallocated. otherwise we have to go chunk by chunk + // + // some kernels are also unable to write into sliced outputs, so + // we respect the kernel's attributes + s.preallocContiguous = + (s.ectx.PreallocContiguous && s.kernel.CanFillSlices() && + s.preallocAllBufs) + + return nil +} + +func (s *scalarExecutor) emitResult(resultData *exec.ArraySpan, data chan<- Datum) error { + var output Datum + if s.allScalars { + // we boxed scalar inputs as ArraySpan so now we have to unbox the output + arr := resultData.MakeArray() + defer arr.Release() + sc, err := scalar.GetScalar(arr, 0) + if err != nil { + return err + } + output = NewDatum(sc) + } else { + d := resultData.MakeData() + defer d.Release() + output = NewDatum(d) + } + data <- output + return nil +} + +func checkAllIsValue(vals []Datum) error { + for _, v := range vals { + if !DatumIsValue(v) { + return fmt.Errorf("%w: tried executing function with non-value type: %s", + arrow.ErrInvalid, v) + } + } + return nil +} + +func checkIfAllScalar(batch *ExecBatch) bool { + for _, v := range batch.Values { + if v.Kind() != KindScalar { + return false + } + } + return batch.NumValues() > 0 +} + +// iterateExecSpans sets up and returns a function which can iterate a batch +// according to the chunk sizes. If the inputs contain chunked arrays, then +// we will find the min(chunk sizes, maxChunkSize) to ensure we return +// contiguous spans to execute on. +// +// the iteration function returns the next span to execute on, the current +// position in the full batch, and a boolean indicating whether or not +// a span was actually returned (there is data to process). +func iterateExecSpans(batch *ExecBatch, maxChunkSize int64, promoteIfAllScalar bool) (haveAllScalars bool, itr spanIterator, err error) { + if batch.NumValues() > 0 { + inferred, allArgsSame := inferBatchLength(batch.Values) + if inferred != batch.Len { + return false, nil, fmt.Errorf("%w: value lengths differed from execbatch length", arrow.ErrInvalid) + } + if !allArgsSame { + return false, nil, fmt.Errorf("%w: array args must all be the same length", arrow.ErrInvalid) + } + } + + var ( + args []Datum = batch.Values + haveChunked bool + chunkIdxes = make([]int, len(args)) + valuePositions = make([]int64, len(args)) + valueOffsets = make([]int64, len(args)) + pos, length int64 = 0, batch.Len + ) + haveAllScalars = checkIfAllScalar(batch) + maxChunkSize = exec.Min(length, maxChunkSize) + + span := exec.ExecSpan{Values: make([]exec.ExecValue, len(args)), Len: 0} + for i, a := range args { + switch arg := a.(type) { + case *ScalarDatum: + span.Values[i].Scalar = arg.Value + case *ArrayDatum: + span.Values[i].Array.SetMembers(arg.Value) + valueOffsets[i] = int64(arg.Value.Offset()) + case *ChunkedDatum: + // populate from first chunk + carr := arg.Value + if len(carr.Chunks()) > 0 { + arr := carr.Chunk(0).Data() + span.Values[i].Array.SetMembers(arr) + valueOffsets[i] = int64(arr.Offset()) + } else { + // fill as zero len + exec.FillZeroLength(carr.DataType(), &span.Values[i].Array) + } + haveChunked = true + } + } + + if haveAllScalars && promoteIfAllScalar { + exec.PromoteExecSpanScalars(span) + } + + nextChunkSpan := func(iterSz int64, span exec.ExecSpan) int64 { + for i := 0; i < len(args) && iterSz > 0; i++ { + // if the argument is not chunked, it's either a scalar or an array + // in which case it doesn't influence the size of the span + chunkedArg, ok := args[i].(*ChunkedDatum) + if !ok { + continue + } + + arg := chunkedArg.Value + if len(arg.Chunks()) == 0 { + iterSz = 0 + continue + } + + var curChunk arrow.Array + for { + curChunk = arg.Chunk(chunkIdxes[i]) + if valuePositions[i] == int64(curChunk.Len()) { + // chunk is zero-length, or was exhausted in the previous + // iteration, move to next chunk + chunkIdxes[i]++ + curChunk = arg.Chunk(chunkIdxes[i]) + span.Values[i].Array.SetMembers(curChunk.Data()) + valuePositions[i] = 0 + valueOffsets[i] = int64(curChunk.Data().Offset()) + continue + } + break + } + iterSz = exec.Min(int64(curChunk.Len())-valuePositions[i], iterSz) + } + return iterSz + } + + return haveAllScalars, func() (exec.ExecSpan, int64, bool) { + if pos == length { + return exec.ExecSpan{}, pos, false + } + + iterationSize := exec.Min(length-pos, maxChunkSize) + if haveChunked { + iterationSize = nextChunkSpan(iterationSize, span) + } + + span.Len = iterationSize + for i, a := range args { + if a.Kind() != KindScalar { + span.Values[i].Array.SetSlice(valuePositions[i]+valueOffsets[i], iterationSize) + valuePositions[i] += iterationSize + } + } + + pos += iterationSize + debug.Assert(pos <= length, "bad state for iteration exec span") + return span, pos, true + }, nil +} + +var ( + // have a pool of scalar executors to avoid excessive object creation + scalarExecPool = sync.Pool{ + New: func() any { return &scalarExecutor{} }, + } +) diff --git a/go/arrow/compute/expression.go b/go/arrow/compute/expression.go index ef6b20b10e87e..e5bd118dab6b7 100644 --- a/go/arrow/compute/expression.go +++ b/go/arrow/compute/expression.go @@ -29,7 +29,7 @@ import ( "github.com/apache/arrow/go/v10/arrow" "github.com/apache/arrow/go/v10/arrow/array" - "github.com/apache/arrow/go/v10/arrow/compute/internal" + "github.com/apache/arrow/go/v10/arrow/compute/internal/exec" "github.com/apache/arrow/go/v10/arrow/internal/debug" "github.com/apache/arrow/go/v10/arrow/ipc" "github.com/apache/arrow/go/v10/arrow/memory" @@ -376,7 +376,7 @@ func (c *Call) Hash() uint64 { h.WriteString(c.funcName) c.cachedHash = h.Sum64() for _, arg := range c.args { - c.cachedHash = internal.HashCombine(c.cachedHash, arg.Hash()) + c.cachedHash = exec.HashCombine(c.cachedHash, arg.Hash()) } return c.cachedHash } diff --git a/go/arrow/compute/functions.go b/go/arrow/compute/functions.go index faa7981ca04e4..c7fd2827b0701 100644 --- a/go/arrow/compute/functions.go +++ b/go/arrow/compute/functions.go @@ -18,6 +18,11 @@ package compute import ( "context" + "fmt" + "strings" + + "github.com/apache/arrow/go/v10/arrow" + "github.com/apache/arrow/go/v10/arrow/compute/internal/exec" ) type Function interface { @@ -27,15 +32,22 @@ type Function interface { Doc() FunctionDoc NumKernels() int Execute(context.Context, FunctionOptions, ...Datum) (Datum, error) + DispatchExact(...arrow.DataType) (exec.Kernel, error) + DispatchBest(...arrow.DataType) (exec.Kernel, error) DefaultOptions() FunctionOptions Validate() error } +// Arity defines the number of required arguments for a function. +// +// Naming conventions are taken from https://en.wikipedia.org/wiki/Arity type Arity struct { NArgs int IsVarArgs bool } +// Convenience functions to generating Arities + func Nullary() Arity { return Arity{0, false} } func Unary() Arity { return Arity{1, false} } func Binary() Arity { return Arity{2, false} } @@ -43,21 +55,254 @@ func Ternary() Arity { return Arity{3, false} } func VarArgs(minArgs int) Arity { return Arity{minArgs, true} } type FunctionDoc struct { - Summary string - Description string - ArgNames []string - OptionsClass string + // A one-line summary of the function, using a verb. + // + // For example, "Add two numeric arrays or scalars" + Summary string + // A detailed description of the function, meant to follow the summary. + Description string + // Symbolic names (identifiers) for the function arguments. + // + // Can be used to generate nicer function signatures. + ArgNames []string + // Name of the options struct type, if any + OptionsType string + // Whether or not options are required for function execution. + // + // If false, then either there are no options for this function, + // or there is a usable default options value. OptionsRequired bool } +// EmptyFuncDoc is a reusable empty function doc definition for convenience. var EmptyFuncDoc FunctionDoc +// FuncKind is an enum representing the type of a function type FuncKind int8 const ( - FuncScalar FuncKind = iota // Scalar - FuncVector // Vector - FuncScalarAgg // ScalarAggregate - FuncHashAgg // HashAggregate - FuncMeta // Meta + // A function that performs scalar data operations on whole arrays + // of data. Can generally process Array or Scalar values. The size + // of the output will be the same as the size (or broadcasted size, + // in the case of mixing Array and Scalar inputs) of the input. + FuncScalar FuncKind = iota // Scalar + // A function with array input and output whose behavior depends on + // the values of the entire arrays passed, rather than the value of + // each scalar value. + FuncVector // Vector + // A function that computes a scalar summary statistic from array input. + FuncScalarAgg // ScalarAggregate + // A function that computes grouped summary statistics from array + // input and an array of group identifiers. + FuncHashAgg // HashAggregate + // A function that dispatches to other functions and does not contain + // its own kernels. + FuncMeta // Meta ) + +func validateFunctionSummary(summary string) error { + if strings.Contains(summary, "\n") { + return fmt.Errorf("%w: summary contains a newline", arrow.ErrInvalid) + } + if summary[len(summary)-1] == '.' { + return fmt.Errorf("%w: summary ends with a point", arrow.ErrInvalid) + } + return nil +} + +func validateFunctionDescription(desc string) error { + if len(desc) != 0 && desc[len(desc)-1] == '\n' { + return fmt.Errorf("%w: description ends with a newline", arrow.ErrInvalid) + } + + const maxLineSize = 78 + for _, ln := range strings.Split(desc, "\n") { + if len(ln) > maxLineSize { + return fmt.Errorf("%w: description line length exceeds %d characters", arrow.ErrInvalid, maxLineSize) + } + } + return nil +} + +// baseFunction is the base class for compute functions. Function +// implementations should embed this baseFunction and will contain +// a collection of "kernels" which are implementations of the function +// for specific argument types. Selecting a viable kernel for +// executing the function is referred to as "dispatching". +type baseFunction struct { + name string + kind FuncKind + arity Arity + doc FunctionDoc + defaultOpts FunctionOptions +} + +func (b *baseFunction) Name() string { return b.name } +func (b *baseFunction) Kind() FuncKind { return b.kind } +func (b *baseFunction) Arity() Arity { return b.arity } +func (b *baseFunction) Doc() FunctionDoc { return b.doc } +func (b *baseFunction) DefaultOptions() FunctionOptions { return b.defaultOpts } +func (b *baseFunction) Validate() error { + if b.doc.Summary == "" { + return nil + } + + argCount := len(b.doc.ArgNames) + if argCount != b.arity.NArgs && !(b.arity.IsVarArgs && argCount == b.arity.NArgs+1) { + return fmt.Errorf("in function '%s': number of argument names for function doc != function arity", b.name) + } + + if err := validateFunctionSummary(b.doc.Summary); err != nil { + return err + } + return validateFunctionDescription(b.doc.Description) +} + +func checkOptions(fn Function, opts FunctionOptions) error { + if opts == nil && fn.Doc().OptionsRequired { + return fmt.Errorf("%w: function '%s' cannot be called without options", arrow.ErrInvalid, fn.Name()) + } + return nil +} + +func (b *baseFunction) checkArity(nargs int) error { + switch { + case b.arity.IsVarArgs && nargs < b.arity.NArgs: + return fmt.Errorf("%w: varargs function '%s' needs at least %d arguments, but only %d passed", + arrow.ErrInvalid, b.name, b.arity.NArgs, nargs) + case !b.arity.IsVarArgs && nargs != b.arity.NArgs: + return fmt.Errorf("%w: function '%s' accepts %d arguments but %d passed", + arrow.ErrInvalid, b.name, b.arity.NArgs, nargs) + } + return nil +} + +// kernelType is a type contstraint interface that is used for funcImpl +// generic definitions. It will be extended as other kernel types +// are defined. +// +// Currently only ScalarKernels are allowed to be used. +type kernelType interface { + exec.ScalarKernel + + // specifying the Kernel interface here allows us to utilize + // the methods of the Kernel interface on the generic + // constrained type + exec.Kernel +} + +// funcImpl is the basic implementation for any functions that use kernels +// i.e. all except for Meta functions. +type funcImpl[KT kernelType] struct { + baseFunction + + kernels []KT +} + +func (fi *funcImpl[KT]) DispatchExact(vals ...arrow.DataType) (*KT, error) { + if err := fi.checkArity(len(vals)); err != nil { + return nil, err + } + + for i := range fi.kernels { + if fi.kernels[i].GetSig().MatchesInputs(vals) { + return &fi.kernels[i], nil + } + } + + return nil, fmt.Errorf("%w: function '%s' has no kernel matching input types %s", + arrow.ErrNotImplemented, fi.name, arrow.TypesToString(vals)) +} + +func (fi *funcImpl[KT]) NumKernels() int { return len(fi.kernels) } +func (fi *funcImpl[KT]) Kernels() []*KT { + res := make([]*KT, len(fi.kernels)) + for i := range fi.kernels { + res[i] = &fi.kernels[i] + } + return res +} + +// A ScalarFunction is a function that executes element-wise operations +// on arrays or scalars, and therefore whose results generally do not +// depent on the order of the values in the arguments. Accepts and returns +// arrays that are all of the same size. These functions roughly correspond +// to the functions used in most SQL expressions. +type ScalarFunction struct { + funcImpl[exec.ScalarKernel] +} + +// NewScalarFunction constructs a new ScalarFunction object with the passed in +// name, arity and function doc. +func NewScalarFunction(name string, arity Arity, doc FunctionDoc) *ScalarFunction { + return &ScalarFunction{ + funcImpl: funcImpl[exec.ScalarKernel]{ + baseFunction: baseFunction{ + name: name, + arity: arity, + doc: doc, + kind: FuncScalar, + }, + }, + } +} + +func (s *ScalarFunction) SetDefaultOptions(opts FunctionOptions) { + s.defaultOpts = opts +} + +func (s *ScalarFunction) DispatchExact(vals ...arrow.DataType) (exec.Kernel, error) { + return s.funcImpl.DispatchExact(vals...) +} + +func (s *ScalarFunction) DispatchBest(vals ...arrow.DataType) (exec.Kernel, error) { + return s.DispatchExact(vals...) +} + +// AddNewKernel constructs a new kernel with the provided signature +// and execution/init functions and then adds it to the function's list of +// kernels. This assumes default null handling (intersection of validity bitmaps) +func (s *ScalarFunction) AddNewKernel(inTypes []exec.InputType, outType exec.OutputType, execFn exec.ArrayKernelExec, init exec.KernelInitFn) error { + if err := s.checkArity(len(inTypes)); err != nil { + return err + } + + if s.arity.IsVarArgs && len(inTypes) != 1 { + return fmt.Errorf("%w: varargs signatures must have exactly one input type", arrow.ErrInvalid) + } + + sig := &exec.KernelSignature{ + InputTypes: inTypes, + OutType: outType, + IsVarArgs: s.arity.IsVarArgs, + } + + s.kernels = append(s.kernels, exec.NewScalarKernelWithSig(sig, execFn, init)) + return nil +} + +// AddKernel adds the provided kernel to the list of kernels +// this function has. A copy of the kernel is added to the slice of kernels, +// which means that a given kernel object can be created, added and then +// reused to add other kernels. +func (s *ScalarFunction) AddKernel(k exec.ScalarKernel) error { + if err := s.checkArity(len(k.Signature.InputTypes)); err != nil { + return err + } + + if s.arity.IsVarArgs && !k.Signature.IsVarArgs { + return fmt.Errorf("%w: function accepts varargs but kernel signature does not", arrow.ErrInvalid) + } + + s.kernels = append(s.kernels, k) + return nil +} + +// Execute uses the passed in context, function options and arguments to eagerly +// execute the function using kernel dispatch, batch iteration and memory +// allocation details as defined by the kernel. +// +// If opts is nil, then the DefaultOptions() will be used. +func (s *ScalarFunction) Execute(ctx context.Context, opts FunctionOptions, args ...Datum) (Datum, error) { + return execInternal(ctx, s, opts, -1, args...) +} diff --git a/go/arrow/compute/go.mod b/go/arrow/compute/go.mod index 447802c46b1d4..09559d7a7a2a0 100644 --- a/go/arrow/compute/go.mod +++ b/go/arrow/compute/go.mod @@ -45,5 +45,6 @@ require ( golang.org/x/mod v0.6.0-dev.0.20220419223038-86c51ed26bb4 // indirect golang.org/x/sys v0.0.0-20220808155132-1c4a2a72c664 // indirect golang.org/x/tools v0.1.12 // indirect + gonum.org/v1/gonum v0.11.0 // indirect gopkg.in/yaml.v3 v3.0.1 // indirect ) diff --git a/go/arrow/compute/internal/hash_util.go b/go/arrow/compute/internal/exec/hash_util.go similarity index 98% rename from go/arrow/compute/internal/hash_util.go rename to go/arrow/compute/internal/exec/hash_util.go index 8fd5cc0448be6..7630c7495f2bc 100644 --- a/go/arrow/compute/internal/hash_util.go +++ b/go/arrow/compute/internal/exec/hash_util.go @@ -14,7 +14,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package internal +package exec import "hash/maphash" diff --git a/go/arrow/compute/internal/kernel.go b/go/arrow/compute/internal/exec/kernel.go similarity index 83% rename from go/arrow/compute/internal/kernel.go rename to go/arrow/compute/internal/exec/kernel.go index 403f27d2335f4..8716db656bb24 100644 --- a/go/arrow/compute/internal/kernel.go +++ b/go/arrow/compute/internal/exec/kernel.go @@ -14,7 +14,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package internal +package exec import ( "context" @@ -26,7 +26,6 @@ import ( "github.com/apache/arrow/go/v10/arrow/bitutil" "github.com/apache/arrow/go/v10/arrow/internal/debug" "github.com/apache/arrow/go/v10/arrow/memory" - "golang.org/x/exp/constraints" "golang.org/x/exp/slices" ) @@ -56,6 +55,17 @@ type Kernel interface { GetSig() *KernelSignature } +// NonAggKernel builds on the base Kernel interface for +// non aggregate execution kernels. Specifically this will +// represent Scalar and Vector kernels. +type NonAggKernel interface { + Kernel + Exec(*KernelCtx, *ExecSpan, *ExecResult) error + GetNullHandling() NullHandling + GetMemAlloc() MemAlloc + CanFillSlices() bool +} + // KernelCtx is a small struct holding the context for a kernel execution // consisting of a pointer to the kernel, initialized state (if needed) // and the context for this execution. @@ -496,7 +506,7 @@ func (k KernelSignature) MatchesInputs(types []arrow.DataType) bool { } for i, t := range types { - if !k.InputTypes[min(i, len(k.InputTypes)-1)].Matches(t) { + if !k.InputTypes[Min(i, len(k.InputTypes)-1)].Matches(t) { return false } } @@ -513,9 +523,65 @@ func (k KernelSignature) MatchesInputs(types []arrow.DataType) bool { return true } -func min[T constraints.Ordered](a, b T) T { - if a < b { - return a +// ArrayKernelExec is an alias definition for a kernel's execution function. +// +// This is used for both stateless and stateful kernels. If a kernel +// depends on some execution state, it can be accessed from the KernelCtx +// object, which also contains the context.Context object which can be +// used for shortcircuiting by checking context.Done / context.Err. +// This allows kernels to control handling timeouts or cancellation of +// computation. +type ArrayKernelExec = func(*KernelCtx, *ExecSpan, *ExecResult) error + +type kernel struct { + Init KernelInitFn + Signature *KernelSignature + Data KernelState + Parallelizable bool +} + +func (k kernel) GetInitFn() KernelInitFn { return k.Init } +func (k kernel) GetSig() *KernelSignature { return k.Signature } + +// A ScalarKernel is the kernel implementation for a Scalar Function. +// In addition to the members found in the base Kernel, it contains +// the null handling and memory pre-allocation preferences. +type ScalarKernel struct { + kernel + + ExecFn ArrayKernelExec + CanWriteIntoSlices bool + NullHandling NullHandling + MemAlloc MemAlloc +} + +// NewScalarKernel constructs a new kernel for scalar execution, constructing +// a KernelSignature with the provided input types and output type, and using +// the passed in execution implementation and initialization function. +func NewScalarKernel(in []InputType, out OutputType, exec ArrayKernelExec, init KernelInitFn) ScalarKernel { + return NewScalarKernelWithSig(&KernelSignature{ + InputTypes: in, + OutType: out, + }, exec, init) +} + +// NewScalarKernelWithSig is a convenience when you already have a signature +// to use for constructing a kernel. It's equivalent to passing the components +// of the signature (input and output types) to NewScalarKernel. +func NewScalarKernelWithSig(sig *KernelSignature, exec ArrayKernelExec, init KernelInitFn) ScalarKernel { + return ScalarKernel{ + kernel: kernel{Signature: sig, Init: init, Parallelizable: true}, + ExecFn: exec, + CanWriteIntoSlices: true, + NullHandling: NullIntersection, + MemAlloc: MemPrealloc, } - return b } + +func (s *ScalarKernel) Exec(ctx *KernelCtx, sp *ExecSpan, out *ExecResult) error { + return s.ExecFn(ctx, sp, out) +} + +func (s ScalarKernel) GetNullHandling() NullHandling { return s.NullHandling } +func (s ScalarKernel) GetMemAlloc() MemAlloc { return s.MemAlloc } +func (s ScalarKernel) CanFillSlices() bool { return s.CanWriteIntoSlices } diff --git a/go/arrow/compute/internal/kernel_test.go b/go/arrow/compute/internal/exec/kernel_test.go similarity index 58% rename from go/arrow/compute/internal/kernel_test.go rename to go/arrow/compute/internal/exec/kernel_test.go index 11d55b0554ad3..3584cfd66b1a9 100644 --- a/go/arrow/compute/internal/kernel_test.go +++ b/go/arrow/compute/internal/exec/kernel_test.go @@ -14,7 +14,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package internal_test +package exec_test import ( "fmt" @@ -23,30 +23,30 @@ import ( "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" + "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/assert" ) func TestTypeMatcherSameTypeID(t *testing.T) { - matcher := internal.SameTypeID(arrow.DECIMAL128) + matcher := exec.SameTypeID(arrow.DECIMAL128) assert.True(t, matcher.Matches(&arrow.Decimal128Type{Precision: 12, Scale: 2})) assert.False(t, matcher.Matches(arrow.PrimitiveTypes.Int8)) assert.Equal(t, "Type::DECIMAL128", matcher.String()) assert.True(t, matcher.Equals(matcher)) - assert.True(t, matcher.Equals(internal.SameTypeID(arrow.DECIMAL))) - assert.False(t, matcher.Equals(internal.SameTypeID(arrow.TIMESTAMP))) - assert.False(t, matcher.Equals(internal.Time32TypeUnit(arrow.Microsecond))) + assert.True(t, matcher.Equals(exec.SameTypeID(arrow.DECIMAL))) + assert.False(t, matcher.Equals(exec.SameTypeID(arrow.TIMESTAMP))) + assert.False(t, matcher.Equals(exec.Time32TypeUnit(arrow.Microsecond))) } func TestTypeMatcherTimestampTypeUnit(t *testing.T) { - matcher := internal.TimestampTypeUnit(arrow.Millisecond) - matcher2 := internal.Time32TypeUnit(arrow.Millisecond) - matcher3 := internal.Time64TypeUnit(arrow.Microsecond) - matcher4 := internal.DurationTypeUnit(arrow.Microsecond) + matcher := exec.TimestampTypeUnit(arrow.Millisecond) + matcher2 := exec.Time32TypeUnit(arrow.Millisecond) + matcher3 := exec.Time64TypeUnit(arrow.Microsecond) + matcher4 := exec.DurationTypeUnit(arrow.Microsecond) assert.True(t, matcher.Matches(arrow.FixedWidthTypes.Timestamp_ms)) assert.True(t, matcher.Matches(&arrow.TimestampType{Unit: arrow.Millisecond, TimeZone: "utc"})) @@ -60,73 +60,73 @@ func TestTypeMatcherTimestampTypeUnit(t *testing.T) { assert.False(t, matcher4.Matches(arrow.FixedWidthTypes.Duration_ms)) // check String() representation - assert.Equal(t, "timestamp(s)", internal.TimestampTypeUnit(arrow.Second).String()) - assert.Equal(t, "timestamp(ms)", internal.TimestampTypeUnit(arrow.Millisecond).String()) - assert.Equal(t, "timestamp(us)", internal.TimestampTypeUnit(arrow.Microsecond).String()) - assert.Equal(t, "timestamp(ns)", internal.TimestampTypeUnit(arrow.Nanosecond).String()) + assert.Equal(t, "timestamp(s)", exec.TimestampTypeUnit(arrow.Second).String()) + assert.Equal(t, "timestamp(ms)", exec.TimestampTypeUnit(arrow.Millisecond).String()) + assert.Equal(t, "timestamp(us)", exec.TimestampTypeUnit(arrow.Microsecond).String()) + assert.Equal(t, "timestamp(ns)", exec.TimestampTypeUnit(arrow.Nanosecond).String()) // equals implementation assert.True(t, matcher.Equals(matcher)) - assert.True(t, matcher.Equals(internal.TimestampTypeUnit(arrow.Millisecond))) - assert.False(t, matcher.Equals(internal.TimestampTypeUnit(arrow.Microsecond))) - assert.False(t, matcher.Equals(internal.Time32TypeUnit(arrow.Millisecond))) + assert.True(t, matcher.Equals(exec.TimestampTypeUnit(arrow.Millisecond))) + assert.False(t, matcher.Equals(exec.TimestampTypeUnit(arrow.Microsecond))) + assert.False(t, matcher.Equals(exec.Time32TypeUnit(arrow.Millisecond))) assert.False(t, matcher3.Equals(matcher2)) assert.False(t, matcher4.Equals(matcher3)) - assert.True(t, matcher4.Equals(internal.DurationTypeUnit(arrow.Microsecond))) - assert.False(t, matcher.Equals(internal.SameTypeID(arrow.TIMESTAMP))) + assert.True(t, matcher4.Equals(exec.DurationTypeUnit(arrow.Microsecond))) + assert.False(t, matcher.Equals(exec.SameTypeID(arrow.TIMESTAMP))) } func TestIntegerMatcher(t *testing.T) { - match := internal.Integer() + match := exec.Integer() assert.Equal(t, "integer", match.String()) assert.True(t, match.Matches(arrow.PrimitiveTypes.Int8)) assert.True(t, match.Matches(arrow.PrimitiveTypes.Uint64)) - assert.True(t, match.Equals(internal.Integer())) - assert.False(t, match.Equals(internal.BinaryLike())) + assert.True(t, match.Equals(exec.Integer())) + assert.False(t, match.Equals(exec.BinaryLike())) } func TestBinaryLikeMatcher(t *testing.T) { - match := internal.BinaryLike() + match := exec.BinaryLike() assert.Equal(t, "binary-like", match.String()) assert.True(t, match.Matches(arrow.BinaryTypes.String)) assert.True(t, match.Matches(arrow.BinaryTypes.Binary)) assert.False(t, match.Matches(arrow.BinaryTypes.LargeString)) assert.False(t, match.Matches(arrow.BinaryTypes.LargeBinary)) - assert.False(t, match.Equals(internal.LargeBinaryLike())) - assert.True(t, match.Equals(internal.BinaryLike())) + assert.False(t, match.Equals(exec.LargeBinaryLike())) + assert.True(t, match.Equals(exec.BinaryLike())) } func TestLargeBinaryLikeMatcher(t *testing.T) { - match := internal.LargeBinaryLike() + match := exec.LargeBinaryLike() assert.Equal(t, "large-binary-like", match.String()) assert.False(t, match.Matches(arrow.BinaryTypes.String)) assert.False(t, match.Matches(arrow.BinaryTypes.Binary)) assert.True(t, match.Matches(arrow.BinaryTypes.LargeString)) assert.True(t, match.Matches(arrow.BinaryTypes.LargeBinary)) - assert.True(t, match.Equals(internal.LargeBinaryLike())) - assert.False(t, match.Equals(internal.BinaryLike())) + assert.True(t, match.Equals(exec.LargeBinaryLike())) + assert.False(t, match.Equals(exec.BinaryLike())) } func TestFixedSizeBinaryMatcher(t *testing.T) { - match := internal.FixedSizeBinaryLike() + match := exec.FixedSizeBinaryLike() assert.Equal(t, "fixed-size-binary-like", match.String()) assert.False(t, match.Matches(arrow.BinaryTypes.String)) assert.True(t, match.Matches(&arrow.Decimal128Type{Precision: 12, Scale: 5})) assert.True(t, match.Matches(&arrow.Decimal256Type{Precision: 12, Scale: 10})) assert.True(t, match.Matches(&arrow.FixedSizeBinaryType{})) - assert.False(t, match.Equals(internal.LargeBinaryLike())) - assert.True(t, match.Equals(internal.FixedSizeBinaryLike())) + assert.False(t, match.Equals(exec.LargeBinaryLike())) + assert.True(t, match.Equals(exec.FixedSizeBinaryLike())) } func TestPrimitiveMatcher(t *testing.T) { - match := internal.Primitive() + match := exec.Primitive() assert.Equal(t, "primitive", match.String()) - assert.True(t, match.Equals(internal.Primitive())) + assert.True(t, match.Equals(exec.Primitive())) types := []arrow.DataType{ arrow.FixedWidthTypes.Boolean, @@ -160,38 +160,38 @@ func TestPrimitiveMatcher(t *testing.T) { } func TestInputTypeAnyType(t *testing.T) { - var ty internal.InputType - assert.Equal(t, internal.InputAny, ty.Kind) + var ty exec.InputType + assert.Equal(t, exec.InputAny, ty.Kind) } func TestInputType(t *testing.T) { - ty1 := internal.NewExactInput(arrow.PrimitiveTypes.Int8) - assert.Equal(t, internal.InputExact, ty1.Kind) + ty1 := exec.NewExactInput(arrow.PrimitiveTypes.Int8) + assert.Equal(t, exec.InputExact, ty1.Kind) assert.True(t, arrow.TypeEqual(arrow.PrimitiveTypes.Int8, ty1.Type)) assert.Equal(t, "int8", ty1.String()) - ty2 := internal.NewIDInput(arrow.DECIMAL) - assert.Equal(t, internal.InputUseMatcher, ty2.Kind) + ty2 := exec.NewIDInput(arrow.DECIMAL) + assert.Equal(t, exec.InputUseMatcher, ty2.Kind) assert.Equal(t, "Type::DECIMAL128", ty2.String()) assert.True(t, ty2.Matcher.Matches(&arrow.Decimal128Type{Precision: 12, Scale: 2})) assert.False(t, ty2.Matcher.Matches(arrow.PrimitiveTypes.Int16)) - ty3 := internal.NewMatchedInput(internal.TimestampTypeUnit(arrow.Microsecond)) + ty3 := exec.NewMatchedInput(exec.TimestampTypeUnit(arrow.Microsecond)) assert.Equal(t, "timestamp(us)", ty3.String()) - var ty4 internal.InputType + var ty4 exec.InputType assert.Equal(t, "any", ty4.String()) // InputAny matches anything assert.True(t, ty4.Matches((arrow.DataType)(nil))) } func TestInputTypeEquals(t *testing.T) { - t1 := internal.NewExactInput(arrow.PrimitiveTypes.Int8) - t2 := internal.NewExactInput(arrow.PrimitiveTypes.Int8) - t3 := internal.NewExactInput(arrow.PrimitiveTypes.Int32) + t1 := exec.NewExactInput(arrow.PrimitiveTypes.Int8) + t2 := exec.NewExactInput(arrow.PrimitiveTypes.Int8) + t3 := exec.NewExactInput(arrow.PrimitiveTypes.Int32) - t5 := internal.NewIDInput(arrow.DECIMAL) - t6 := internal.NewIDInput(arrow.DECIMAL) + t5 := exec.NewIDInput(arrow.DECIMAL) + t6 := exec.NewIDInput(arrow.DECIMAL) assert.True(t, t1.Equals(&t2)) assert.False(t, t1.Equals(&t3)) @@ -199,26 +199,26 @@ func TestInputTypeEquals(t *testing.T) { assert.True(t, t5.Equals(&t5)) assert.True(t, t5.Equals(&t6)) - var ty internal.InputType - assert.True(t, ty.Equals(&internal.InputType{Kind: internal.InputAny})) + var ty exec.InputType + assert.True(t, ty.Equals(&exec.InputType{Kind: exec.InputAny})) // for now, an ID matcher for arrow.INT32 and a ExactInput for // arrow.PrimitiveTypes.Int32 are treated as being different. // this could be made equivalent later if desireable // check that field metadata is excluded from equality checks - t7 := internal.NewExactInput(arrow.ListOfField( + t7 := exec.NewExactInput(arrow.ListOfField( arrow.Field{Name: "item", Type: arrow.BinaryTypes.String, Nullable: true, Metadata: arrow.NewMetadata([]string{"foo"}, []string{"bar"})})) - t8 := internal.NewExactInput(arrow.ListOf(arrow.BinaryTypes.String)) + t8 := exec.NewExactInput(arrow.ListOf(arrow.BinaryTypes.String)) assert.True(t, t7.Equals(&t8)) } func TestInputTypeHash(t *testing.T) { var ( - t0 internal.InputType - t1 = internal.NewExactInput(arrow.PrimitiveTypes.Int8) - t2 = internal.NewIDInput(arrow.DECIMAL) + t0 exec.InputType + t1 = exec.NewExactInput(arrow.PrimitiveTypes.Int8) + t2 = exec.NewIDInput(arrow.DECIMAL) ) // these checks try to determine first of all whether hash @@ -233,12 +233,12 @@ func TestInputTypeHash(t *testing.T) { } func TestInputTypeMatches(t *testing.T) { - in1 := internal.NewExactInput(arrow.PrimitiveTypes.Int8) + in1 := exec.NewExactInput(arrow.PrimitiveTypes.Int8) assert.True(t, in1.Matches(arrow.PrimitiveTypes.Int8)) assert.False(t, in1.Matches(arrow.PrimitiveTypes.Int16)) - in2 := internal.NewIDInput(arrow.DECIMAL) + in2 := exec.NewIDInput(arrow.DECIMAL) assert.True(t, in2.Matches(&arrow.Decimal128Type{Precision: 12, Scale: 2})) ty2 := &arrow.Decimal128Type{Precision: 12, Scale: 2} @@ -265,27 +265,27 @@ func TestInputTypeMatches(t *testing.T) { } func TestOutputType(t *testing.T) { - ty1 := internal.NewOutputType(arrow.PrimitiveTypes.Int8) - assert.Equal(t, internal.ResolveFixed, ty1.Kind) + ty1 := exec.NewOutputType(arrow.PrimitiveTypes.Int8) + assert.Equal(t, exec.ResolveFixed, ty1.Kind) assert.True(t, arrow.TypeEqual(arrow.PrimitiveTypes.Int8, ty1.Type)) - dummyResolver := func(_ *internal.KernelCtx, args []arrow.DataType) (arrow.DataType, error) { + dummyResolver := func(_ *exec.KernelCtx, args []arrow.DataType) (arrow.DataType, error) { return arrow.PrimitiveTypes.Int32, nil } - ty2 := internal.NewComputedOutputType(dummyResolver) - assert.Equal(t, internal.ResolveComputed, ty2.Kind) + ty2 := exec.NewComputedOutputType(dummyResolver) + assert.Equal(t, exec.ResolveComputed, ty2.Kind) outType2, err := ty2.Resolve(nil, nil) assert.NoError(t, err) assert.Same(t, arrow.PrimitiveTypes.Int32, outType2) ty3 := ty1 - assert.Equal(t, internal.ResolveFixed, ty3.Kind) + assert.Equal(t, exec.ResolveFixed, ty3.Kind) assert.True(t, arrow.TypeEqual(ty1.Type, ty3.Type)) ty4 := ty2 - assert.Equal(t, internal.ResolveComputed, ty4.Kind) + assert.Equal(t, exec.ResolveComputed, ty4.Kind) outType4, err := ty4.Resolve(nil, nil) assert.NoError(t, err) assert.Same(t, arrow.PrimitiveTypes.Int32, outType4) @@ -295,7 +295,7 @@ func TestOutputType(t *testing.T) { } func TestOutputTypeResolve(t *testing.T) { - ty1 := internal.NewOutputType(arrow.PrimitiveTypes.Int32) + ty1 := exec.NewOutputType(arrow.PrimitiveTypes.Int32) result, err := ty1.Resolve(nil, nil) assert.NoError(t, err) @@ -309,17 +309,17 @@ func TestOutputTypeResolve(t *testing.T) { assert.NoError(t, err) assert.Same(t, arrow.PrimitiveTypes.Int32, result) - resolver := func(_ *internal.KernelCtx, args []arrow.DataType) (arrow.DataType, error) { + resolver := func(_ *exec.KernelCtx, args []arrow.DataType) (arrow.DataType, error) { return args[0], nil } - ty2 := internal.NewComputedOutputType(resolver) + ty2 := exec.NewComputedOutputType(resolver) result, err = ty2.Resolve(nil, []arrow.DataType{arrow.BinaryTypes.String}) assert.NoError(t, err) assert.Same(t, arrow.BinaryTypes.String, result) // type resolver that returns an error - ty3 := internal.NewComputedOutputType(func(_ *internal.KernelCtx, dt []arrow.DataType) (arrow.DataType, error) { + ty3 := exec.NewComputedOutputType(func(_ *exec.KernelCtx, dt []arrow.DataType) (arrow.DataType, error) { // checking the value types versus the function arity should be validated // elsewhere. this is just for illustration purposes if len(dt) == 0 { @@ -332,7 +332,7 @@ func TestOutputTypeResolve(t *testing.T) { assert.ErrorIs(t, err, arrow.ErrInvalid) // resolver returns a fixed value - ty4 := internal.NewComputedOutputType(func(*internal.KernelCtx, []arrow.DataType) (arrow.DataType, error) { + ty4 := exec.NewComputedOutputType(func(*exec.KernelCtx, []arrow.DataType) (arrow.DataType, error) { return arrow.PrimitiveTypes.Int32, nil }) result, err = ty4.Resolve(nil, []arrow.DataType{arrow.PrimitiveTypes.Int8}) @@ -344,46 +344,46 @@ func TestOutputTypeResolve(t *testing.T) { } func TestKernelSignatureEquals(t *testing.T) { - sig1 := internal.KernelSignature{ - InputTypes: []internal.InputType{}, - OutType: internal.NewOutputType(arrow.BinaryTypes.String)} - sig1Copy := internal.KernelSignature{ - InputTypes: []internal.InputType{}, - OutType: internal.NewOutputType(arrow.BinaryTypes.String)} - sig2 := internal.KernelSignature{ - InputTypes: []internal.InputType{ - internal.NewExactInput(arrow.PrimitiveTypes.Int8)}, - OutType: internal.NewOutputType(arrow.BinaryTypes.String), + sig1 := exec.KernelSignature{ + InputTypes: []exec.InputType{}, + OutType: exec.NewOutputType(arrow.BinaryTypes.String)} + sig1Copy := exec.KernelSignature{ + InputTypes: []exec.InputType{}, + OutType: exec.NewOutputType(arrow.BinaryTypes.String)} + sig2 := exec.KernelSignature{ + InputTypes: []exec.InputType{ + exec.NewExactInput(arrow.PrimitiveTypes.Int8)}, + OutType: exec.NewOutputType(arrow.BinaryTypes.String), } // output type doesn't matter (for now) - sig3 := internal.KernelSignature{ - InputTypes: []internal.InputType{ - internal.NewExactInput(arrow.PrimitiveTypes.Int8)}, - OutType: internal.NewOutputType(arrow.PrimitiveTypes.Int32), + sig3 := exec.KernelSignature{ + InputTypes: []exec.InputType{ + exec.NewExactInput(arrow.PrimitiveTypes.Int8)}, + OutType: exec.NewOutputType(arrow.PrimitiveTypes.Int32), } - sig4 := internal.KernelSignature{ - InputTypes: []internal.InputType{ - internal.NewExactInput(arrow.PrimitiveTypes.Int8), - internal.NewExactInput(arrow.PrimitiveTypes.Int16), + sig4 := exec.KernelSignature{ + InputTypes: []exec.InputType{ + exec.NewExactInput(arrow.PrimitiveTypes.Int8), + exec.NewExactInput(arrow.PrimitiveTypes.Int16), }, - OutType: internal.NewOutputType(arrow.BinaryTypes.String), + OutType: exec.NewOutputType(arrow.BinaryTypes.String), } - sig4Copy := internal.KernelSignature{ - InputTypes: []internal.InputType{ - internal.NewExactInput(arrow.PrimitiveTypes.Int8), - internal.NewExactInput(arrow.PrimitiveTypes.Int16), + sig4Copy := exec.KernelSignature{ + InputTypes: []exec.InputType{ + exec.NewExactInput(arrow.PrimitiveTypes.Int8), + exec.NewExactInput(arrow.PrimitiveTypes.Int16), }, - OutType: internal.NewOutputType(arrow.BinaryTypes.String), + OutType: exec.NewOutputType(arrow.BinaryTypes.String), } - sig5 := internal.KernelSignature{ - InputTypes: []internal.InputType{ - internal.NewExactInput(arrow.PrimitiveTypes.Int8), - internal.NewExactInput(arrow.PrimitiveTypes.Int16), - internal.NewExactInput(arrow.PrimitiveTypes.Int32), + sig5 := exec.KernelSignature{ + InputTypes: []exec.InputType{ + exec.NewExactInput(arrow.PrimitiveTypes.Int8), + exec.NewExactInput(arrow.PrimitiveTypes.Int16), + exec.NewExactInput(arrow.PrimitiveTypes.Int32), }, - OutType: internal.NewOutputType(arrow.BinaryTypes.String), + OutType: exec.NewOutputType(arrow.BinaryTypes.String), } assert.True(t, sig1.Equals(sig1)) @@ -399,19 +399,19 @@ func TestKernelSignatureEquals(t *testing.T) { } func TestKernelSignatureVarArgsEqual(t *testing.T) { - sig1 := internal.KernelSignature{ - InputTypes: []internal.InputType{internal.NewExactInput(arrow.PrimitiveTypes.Int8)}, - OutType: internal.NewOutputType(arrow.BinaryTypes.String), + sig1 := exec.KernelSignature{ + InputTypes: []exec.InputType{exec.NewExactInput(arrow.PrimitiveTypes.Int8)}, + OutType: exec.NewOutputType(arrow.BinaryTypes.String), IsVarArgs: true, } - sig2 := internal.KernelSignature{ - InputTypes: []internal.InputType{internal.NewExactInput(arrow.PrimitiveTypes.Int8)}, - OutType: internal.NewOutputType(arrow.BinaryTypes.String), + sig2 := exec.KernelSignature{ + InputTypes: []exec.InputType{exec.NewExactInput(arrow.PrimitiveTypes.Int8)}, + OutType: exec.NewOutputType(arrow.BinaryTypes.String), IsVarArgs: true, } - sig3 := internal.KernelSignature{ - InputTypes: []internal.InputType{internal.NewExactInput(arrow.PrimitiveTypes.Int8)}, - OutType: internal.NewOutputType(arrow.BinaryTypes.String), + sig3 := exec.KernelSignature{ + InputTypes: []exec.InputType{exec.NewExactInput(arrow.PrimitiveTypes.Int8)}, + OutType: exec.NewOutputType(arrow.BinaryTypes.String), } assert.True(t, sig1.Equals(sig2)) @@ -419,19 +419,19 @@ func TestKernelSignatureVarArgsEqual(t *testing.T) { } func TestKernelSignatureHash(t *testing.T) { - sig1 := internal.KernelSignature{ - InputTypes: []internal.InputType{}, - OutType: internal.NewOutputType(arrow.BinaryTypes.String), + sig1 := exec.KernelSignature{ + InputTypes: []exec.InputType{}, + OutType: exec.NewOutputType(arrow.BinaryTypes.String), } - sig2 := internal.KernelSignature{ - InputTypes: []internal.InputType{internal.NewExactInput(arrow.PrimitiveTypes.Int8)}, - OutType: internal.NewOutputType(arrow.BinaryTypes.String), + sig2 := exec.KernelSignature{ + InputTypes: []exec.InputType{exec.NewExactInput(arrow.PrimitiveTypes.Int8)}, + OutType: exec.NewOutputType(arrow.BinaryTypes.String), } - sig3 := internal.KernelSignature{ - InputTypes: []internal.InputType{ - internal.NewExactInput(arrow.PrimitiveTypes.Int8), - internal.NewExactInput(arrow.PrimitiveTypes.Int32)}, - OutType: internal.NewOutputType(arrow.BinaryTypes.String), + sig3 := exec.KernelSignature{ + InputTypes: []exec.InputType{ + exec.NewExactInput(arrow.PrimitiveTypes.Int8), + exec.NewExactInput(arrow.PrimitiveTypes.Int32)}, + OutType: exec.NewOutputType(arrow.BinaryTypes.String), } assert.Equal(t, sig1.Hash(), sig1.Hash()) @@ -442,18 +442,18 @@ func TestKernelSignatureHash(t *testing.T) { func TestKernelSignatureMatchesInputs(t *testing.T) { // () -> boolean - sig1 := internal.KernelSignature{ - OutType: internal.NewOutputType(arrow.FixedWidthTypes.Boolean)} + sig1 := exec.KernelSignature{ + OutType: exec.NewOutputType(arrow.FixedWidthTypes.Boolean)} assert.True(t, sig1.MatchesInputs([]arrow.DataType{})) assert.False(t, sig1.MatchesInputs([]arrow.DataType{arrow.PrimitiveTypes.Int8})) // (int8, decimal) -> boolean - sig2 := internal.KernelSignature{ - InputTypes: []internal.InputType{ - internal.NewExactInput(arrow.PrimitiveTypes.Int8), - internal.NewIDInput(arrow.DECIMAL)}, - OutType: internal.NewOutputType(arrow.FixedWidthTypes.Boolean), + sig2 := exec.KernelSignature{ + InputTypes: []exec.InputType{ + exec.NewExactInput(arrow.PrimitiveTypes.Int8), + exec.NewIDInput(arrow.DECIMAL)}, + OutType: exec.NewOutputType(arrow.FixedWidthTypes.Boolean), } assert.False(t, sig2.MatchesInputs([]arrow.DataType{})) assert.False(t, sig2.MatchesInputs([]arrow.DataType{arrow.PrimitiveTypes.Int8})) @@ -462,12 +462,12 @@ func TestKernelSignatureMatchesInputs(t *testing.T) { &arrow.Decimal128Type{Precision: 12, Scale: 2}})) // (int8, int32) -> boolean - sig3 := internal.KernelSignature{ - InputTypes: []internal.InputType{ - internal.NewExactInput(arrow.PrimitiveTypes.Int8), - internal.NewExactInput(arrow.PrimitiveTypes.Int32), + sig3 := exec.KernelSignature{ + InputTypes: []exec.InputType{ + exec.NewExactInput(arrow.PrimitiveTypes.Int8), + exec.NewExactInput(arrow.PrimitiveTypes.Int32), }, - OutType: internal.NewOutputType(arrow.FixedWidthTypes.Boolean), + OutType: exec.NewOutputType(arrow.FixedWidthTypes.Boolean), } assert.False(t, sig3.MatchesInputs(nil)) assert.True(t, sig3.MatchesInputs([]arrow.DataType{arrow.PrimitiveTypes.Int8, arrow.PrimitiveTypes.Int32})) @@ -476,9 +476,9 @@ func TestKernelSignatureMatchesInputs(t *testing.T) { func TestKernelSignatureVarArgsMatchesInputs(t *testing.T) { { - sig := internal.KernelSignature{ - InputTypes: []internal.InputType{internal.NewExactInput(arrow.PrimitiveTypes.Int8)}, - OutType: internal.NewOutputType(arrow.BinaryTypes.String), + sig := exec.KernelSignature{ + InputTypes: []exec.InputType{exec.NewExactInput(arrow.PrimitiveTypes.Int8)}, + OutType: exec.NewOutputType(arrow.BinaryTypes.String), IsVarArgs: true, } @@ -490,12 +490,12 @@ func TestKernelSignatureVarArgsMatchesInputs(t *testing.T) { assert.False(t, sig.MatchesInputs(args)) } { - sig := internal.KernelSignature{ - InputTypes: []internal.InputType{ - internal.NewExactInput(arrow.PrimitiveTypes.Int8), - internal.NewExactInput(arrow.BinaryTypes.String), + sig := exec.KernelSignature{ + InputTypes: []exec.InputType{ + exec.NewExactInput(arrow.PrimitiveTypes.Int8), + exec.NewExactInput(arrow.BinaryTypes.String), }, - OutType: internal.NewOutputType(arrow.BinaryTypes.String), + OutType: exec.NewOutputType(arrow.BinaryTypes.String), IsVarArgs: true, } @@ -509,43 +509,43 @@ func TestKernelSignatureVarArgsMatchesInputs(t *testing.T) { } func TestKernelSignatureToString(t *testing.T) { - inTypes := []internal.InputType{ - internal.NewExactInput(arrow.PrimitiveTypes.Int8), - internal.NewIDInput(arrow.DECIMAL), - internal.NewExactInput(arrow.BinaryTypes.String), + inTypes := []exec.InputType{ + exec.NewExactInput(arrow.PrimitiveTypes.Int8), + exec.NewIDInput(arrow.DECIMAL), + exec.NewExactInput(arrow.BinaryTypes.String), } - sig := internal.KernelSignature{ - InputTypes: inTypes, OutType: internal.NewOutputType(arrow.BinaryTypes.String), + sig := exec.KernelSignature{ + InputTypes: inTypes, OutType: exec.NewOutputType(arrow.BinaryTypes.String), } assert.Equal(t, "(int8, Type::DECIMAL128, utf8) -> utf8", sig.String()) - outType := internal.NewComputedOutputType(func(*internal.KernelCtx, []arrow.DataType) (arrow.DataType, error) { + outType := exec.NewComputedOutputType(func(*exec.KernelCtx, []arrow.DataType) (arrow.DataType, error) { return nil, arrow.ErrInvalid }) - sig2 := internal.KernelSignature{ - InputTypes: []internal.InputType{ - internal.NewExactInput(arrow.PrimitiveTypes.Int8), - internal.NewIDInput(arrow.DECIMAL)}, + sig2 := exec.KernelSignature{ + InputTypes: []exec.InputType{ + exec.NewExactInput(arrow.PrimitiveTypes.Int8), + exec.NewIDInput(arrow.DECIMAL)}, OutType: outType, } assert.Equal(t, "(int8, Type::DECIMAL128) -> computed", sig2.String()) } func TestKernelSignatureVarArgsToString(t *testing.T) { - sig1 := internal.KernelSignature{ - InputTypes: []internal.InputType{ - internal.NewExactInput(arrow.PrimitiveTypes.Int8)}, - OutType: internal.NewOutputType(arrow.BinaryTypes.String), + sig1 := exec.KernelSignature{ + InputTypes: []exec.InputType{ + exec.NewExactInput(arrow.PrimitiveTypes.Int8)}, + OutType: exec.NewOutputType(arrow.BinaryTypes.String), IsVarArgs: true, } assert.Equal(t, "varargs[int8*] -> utf8", sig1.String()) - sig2 := internal.KernelSignature{ - InputTypes: []internal.InputType{ - internal.NewExactInput(arrow.BinaryTypes.String), - internal.NewExactInput(arrow.PrimitiveTypes.Int8)}, - OutType: internal.NewOutputType(arrow.BinaryTypes.String), + sig2 := exec.KernelSignature{ + InputTypes: []exec.InputType{ + exec.NewExactInput(arrow.BinaryTypes.String), + exec.NewExactInput(arrow.PrimitiveTypes.Int8)}, + OutType: exec.NewOutputType(arrow.BinaryTypes.String), IsVarArgs: true, } assert.Equal(t, "varargs[utf8, int8*] -> utf8", sig2.String()) diff --git a/go/arrow/compute/internal/span.go b/go/arrow/compute/internal/exec/span.go similarity index 97% rename from go/arrow/compute/internal/span.go rename to go/arrow/compute/internal/exec/span.go index 8f4a43f3b9500..2f4e7330d129d 100644 --- a/go/arrow/compute/internal/span.go +++ b/go/arrow/compute/internal/exec/span.go @@ -14,7 +14,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package internal +package exec import ( "reflect" @@ -314,7 +314,7 @@ func (a *ArraySpan) FillFromScalar(val scalar.Scalar) { } else { // even when the value is null, we must populate // child data to yield a valid array. ugh - fillZeroLength(sc.DataType().(arrow.NestedType).Fields()[0].Type, &a.Children[0]) + FillZeroLength(sc.DataType().(arrow.NestedType).Fields()[0].Type, &a.Children[0]) } switch typeID { @@ -365,7 +365,7 @@ func (a *ArraySpan) FillFromScalar(val scalar.Scalar) { if i == childIDS[sc.TypeCode] { a.Children[i].FillFromScalar(sc.Value) } else { - fillZeroLength(f.Type, &a.Children[i]) + FillZeroLength(f.Type, &a.Children[i]) } } case *scalar.SparseUnion: @@ -496,7 +496,9 @@ func getNumBuffers(dt arrow.DataType) int { } } -func fillZeroLength(dt arrow.DataType, span *ArraySpan) { +// FillZeroLength fills an ArraySpan with the appropriate information for +// a Zero Length Array of the provided type. +func FillZeroLength(dt arrow.DataType, span *ArraySpan) { span.Scratch[0], span.Scratch[1] = 0, 0 span.Type = dt span.Len = 0 @@ -524,7 +526,7 @@ func fillZeroLength(dt arrow.DataType, span *ArraySpan) { span.Children = make([]ArraySpan, len(nt.Fields())) } for i, f := range nt.Fields() { - fillZeroLength(f.Type, &span.Children[i]) + FillZeroLength(f.Type, &span.Children[i]) } } diff --git a/go/arrow/compute/internal/span_test.go b/go/arrow/compute/internal/exec/span_test.go similarity index 87% rename from go/arrow/compute/internal/span_test.go rename to go/arrow/compute/internal/exec/span_test.go index 1c55a7e55edbd..9187dae5d51be 100644 --- a/go/arrow/compute/internal/span_test.go +++ b/go/arrow/compute/internal/exec/span_test.go @@ -14,7 +14,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package internal_test +package exec_test import ( "reflect" @@ -24,7 +24,7 @@ import ( "github.com/apache/arrow/go/v10/arrow" "github.com/apache/arrow/go/v10/arrow/array" - "github.com/apache/arrow/go/v10/arrow/compute/internal" + "github.com/apache/arrow/go/v10/arrow/compute/internal/exec" "github.com/apache/arrow/go/v10/arrow/decimal128" "github.com/apache/arrow/go/v10/arrow/endian" "github.com/apache/arrow/go/v10/arrow/internal/testing/types" @@ -53,7 +53,7 @@ func TestBufferSpan_SetBuffer(t *testing.T) { } for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - b := &internal.BufferSpan{ + b := &exec.BufferSpan{ Buf: tt.fields.Buf, Owner: tt.fields.Owner, SelfAlloc: tt.fields.SelfAlloc, @@ -86,7 +86,7 @@ func TestBufferSpan_WrapBuffer(t *testing.T) { } for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - b := &internal.BufferSpan{ + b := &exec.BufferSpan{ Buf: tt.fields.Buf, Owner: tt.fields.Owner, SelfAlloc: tt.fields.SelfAlloc, @@ -105,9 +105,9 @@ func TestArraySpan_UpdateNullCount(t *testing.T) { Len int64 Nulls int64 Offset int64 - Buffers [3]internal.BufferSpan + Buffers [3]exec.BufferSpan Scratch [2]uint64 - Children []internal.ArraySpan + Children []exec.ArraySpan } tests := []struct { name string @@ -118,16 +118,16 @@ func TestArraySpan_UpdateNullCount(t *testing.T) { {"unknown", fields{ Nulls: array.UnknownNullCount, Len: 8, // 0b01101101 - Buffers: [3]internal.BufferSpan{{Buf: []byte{109}}, {}, {}}}, 3}, + Buffers: [3]exec.BufferSpan{{Buf: []byte{109}}, {}, {}}}, 3}, {"unknown with offset", fields{ Nulls: array.UnknownNullCount, Len: 4, Offset: 2, // 0b01101101 - Buffers: [3]internal.BufferSpan{{Buf: []byte{109}}, {}, {}}}, 1}, + Buffers: [3]exec.BufferSpan{{Buf: []byte{109}}, {}, {}}}, 1}, } for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - a := &internal.ArraySpan{ + a := &exec.ArraySpan{ Type: tt.fields.Type, Len: tt.fields.Len, Nulls: tt.fields.Nulls, @@ -149,21 +149,21 @@ func TestArraySpan_Dictionary(t *testing.T) { Len int64 Nulls int64 Offset int64 - Buffers [3]internal.BufferSpan + Buffers [3]exec.BufferSpan Scratch [2]uint64 - Children []internal.ArraySpan + Children []exec.ArraySpan } - children := []internal.ArraySpan{{}} + children := []exec.ArraySpan{{}} tests := []struct { name string fields fields - want *internal.ArraySpan + want *exec.ArraySpan }{ {"basic", fields{Children: children}, &children[0]}, } for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - a := &internal.ArraySpan{ + a := &exec.ArraySpan{ Type: tt.fields.Type, Len: tt.fields.Len, Nulls: tt.fields.Nulls, @@ -185,9 +185,9 @@ func TestArraySpan_NumBuffers(t *testing.T) { Len int64 Nulls int64 Offset int64 - Buffers [3]internal.BufferSpan + Buffers [3]exec.BufferSpan Scratch [2]uint64 - Children []internal.ArraySpan + Children []exec.ArraySpan } arrow.RegisterExtensionType(types.NewUUIDType()) @@ -210,7 +210,7 @@ func TestArraySpan_NumBuffers(t *testing.T) { } for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - a := &internal.ArraySpan{ + a := &exec.ArraySpan{ Type: tt.fields.Type, Len: tt.fields.Len, Nulls: tt.fields.Nulls, @@ -232,9 +232,9 @@ func TestArraySpan_MakeData(t *testing.T) { Len int64 Nulls int64 Offset int64 - Buffers [3]internal.BufferSpan + Buffers [3]exec.BufferSpan Scratch [2]uint64 - Children []internal.ArraySpan + Children []exec.ArraySpan } var ( @@ -307,7 +307,7 @@ func TestArraySpan_MakeData(t *testing.T) { ret := fields{ Type: arrow.ListOf(arrow.PrimitiveTypes.Int8), Len: 1, - Children: []internal.ArraySpan{{ + Children: []exec.ArraySpan{{ Type: arrow.PrimitiveTypes.Int8, Len: 4, }}, @@ -350,7 +350,7 @@ func TestArraySpan_MakeData(t *testing.T) { ret := fields{ Type: types.NewDictExtensionType(), Len: 1, - Children: []internal.ArraySpan{{ + Children: []exec.ArraySpan{{ Type: arrow.BinaryTypes.String, Len: 2, }}, @@ -395,7 +395,7 @@ func TestArraySpan_MakeData(t *testing.T) { t.Run("MakeData", func(t *testing.T) { f := tt.fields(mem) - a := &internal.ArraySpan{ + a := &exec.ArraySpan{ Type: f.Type, Len: f.Len, Nulls: f.Nulls, @@ -415,7 +415,7 @@ func TestArraySpan_MakeData(t *testing.T) { t.Run("MakeArray", func(t *testing.T) { f := tt.fields(mem) - a := &internal.ArraySpan{ + a := &exec.ArraySpan{ Type: f.Type, Len: f.Len, Nulls: f.Nulls, @@ -444,9 +444,9 @@ func TestArraySpan_SetSlice(t *testing.T) { Len int64 Nulls int64 Offset int64 - Buffers [3]internal.BufferSpan + Buffers [3]exec.BufferSpan Scratch [2]uint64 - Children []internal.ArraySpan + Children []exec.ArraySpan } type args struct { off int64 @@ -463,7 +463,7 @@ func TestArraySpan_SetSlice(t *testing.T) { } for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - a := &internal.ArraySpan{ + a := &exec.ArraySpan{ Type: tt.fields.Type, Len: tt.fields.Len, Nulls: tt.fields.Nulls, @@ -495,56 +495,56 @@ func TestArraySpan_FillFromScalar(t *testing.T) { tests := []struct { name string args scalar.Scalar - exp internal.ArraySpan + exp exec.ArraySpan }{ {"null-type", scalar.MakeNullScalar(arrow.Null), - internal.ArraySpan{Type: arrow.Null, Len: 1, Nulls: 1}}, + exec.ArraySpan{Type: arrow.Null, Len: 1, Nulls: 1}}, {"bool valid", scalar.MakeScalar(true), - internal.ArraySpan{ + exec.ArraySpan{ Type: arrow.FixedWidthTypes.Boolean, Len: 1, Nulls: 0, - Buffers: [3]internal.BufferSpan{{Buf: []byte{0x01}}, {Buf: []byte{0x01}}, {}}, + Buffers: [3]exec.BufferSpan{{Buf: []byte{0x01}}, {Buf: []byte{0x01}}, {}}, }}, {"bool valid false", scalar.MakeScalar(false), - internal.ArraySpan{ + exec.ArraySpan{ Type: arrow.FixedWidthTypes.Boolean, Len: 1, Nulls: 0, - Buffers: [3]internal.BufferSpan{{Buf: []byte{0x01}}, {Buf: []byte{0x00}}, {}}, + Buffers: [3]exec.BufferSpan{{Buf: []byte{0x01}}, {Buf: []byte{0x00}}, {}}, }}, {"primitive null", scalar.MakeNullScalar(arrow.PrimitiveTypes.Int32), - internal.ArraySpan{ + exec.ArraySpan{ Type: arrow.PrimitiveTypes.Int32, Len: 1, Nulls: 1, - Buffers: [3]internal.BufferSpan{{Buf: []byte{0x00}}, {Buf: []byte{0, 0, 0, 0}}, {}}, + Buffers: [3]exec.BufferSpan{{Buf: []byte{0x00}}, {Buf: []byte{0, 0, 0, 0}}, {}}, }}, {"decimal valid", scalar.NewDecimal128Scalar(decimal128.FromU64(1234), &arrow.Decimal128Type{Precision: 12, Scale: 2}), - internal.ArraySpan{ + exec.ArraySpan{ Type: &arrow.Decimal128Type{Precision: 12, Scale: 2}, Len: 1, Nulls: 0, - Buffers: [3]internal.BufferSpan{{Buf: []byte{0x01}}, {Buf: expDecimalBuf[:]}, {}}, + Buffers: [3]exec.BufferSpan{{Buf: []byte{0x01}}, {Buf: expDecimalBuf[:]}, {}}, }}, {"dictionary scalar", scalar.NewDictScalar(scalar.NewInt8Scalar(1), dict), - internal.ArraySpan{ + exec.ArraySpan{ Type: &arrow.DictionaryType{IndexType: arrow.PrimitiveTypes.Int8, ValueType: arrow.BinaryTypes.String}, Len: 1, Nulls: 0, - Buffers: [3]internal.BufferSpan{{Buf: []byte{0x01}}, + Buffers: [3]exec.BufferSpan{{Buf: []byte{0x01}}, {Buf: []byte{1}}, {}, }, - Children: []internal.ArraySpan{{ + Children: []exec.ArraySpan{{ Type: arrow.BinaryTypes.String, Len: 2, - Buffers: [3]internal.BufferSpan{ + Buffers: [3]exec.BufferSpan{ {Buf: dict.NullBitmapBytes(), Owner: dict.Data().Buffers()[0]}, {Buf: dict.Data().Buffers()[1].Bytes(), Owner: dict.Data().Buffers()[1]}, {Buf: dict.Data().Buffers()[2].Bytes(), Owner: dict.Data().Buffers()[2]}, @@ -554,12 +554,12 @@ func TestArraySpan_FillFromScalar(t *testing.T) { }, {"binary scalar", scalar.NewBinaryScalar(dict.Data().Buffers()[2], arrow.BinaryTypes.String), - internal.ArraySpan{ + exec.ArraySpan{ Type: arrow.BinaryTypes.String, Len: 1, Nulls: 0, Scratch: expScratch, - Buffers: [3]internal.BufferSpan{ + Buffers: [3]exec.BufferSpan{ {Buf: []byte{0x01}}, {Buf: arrow.Uint64Traits.CastToBytes(expScratch[:1])}, {Buf: dict.Data().Buffers()[2].Bytes(), Owner: dict.Data().Buffers()[2]}}, @@ -567,55 +567,55 @@ func TestArraySpan_FillFromScalar(t *testing.T) { }, {"large binary", scalar.NewLargeStringScalarFromBuffer(dict.Data().Buffers()[2]), - internal.ArraySpan{ + exec.ArraySpan{ Type: arrow.BinaryTypes.LargeString, Len: 1, Nulls: 0, Scratch: [2]uint64{0, 10}, - Buffers: [3]internal.BufferSpan{ + Buffers: [3]exec.BufferSpan{ {Buf: []byte{0x01}}, {Buf: arrow.Uint64Traits.CastToBytes([]uint64{0, 10})}, {Buf: dict.Data().Buffers()[2].Bytes(), Owner: dict.Data().Buffers()[2]}}, }}, {"fixed size binary", scalar.NewFixedSizeBinaryScalar(dict.Data().Buffers()[2], &arrow.FixedSizeBinaryType{ByteWidth: 10}), - internal.ArraySpan{ + exec.ArraySpan{ Type: &arrow.FixedSizeBinaryType{ByteWidth: 10}, Len: 1, - Buffers: [3]internal.BufferSpan{ + Buffers: [3]exec.BufferSpan{ {Buf: []byte{0x01}}, {Buf: dict.Data().Buffers()[2].Bytes(), Owner: dict.Data().Buffers()[2]}, {}, }, }}, {"map scalar null value", scalar.MakeNullScalar(arrow.MapOf(arrow.PrimitiveTypes.Int8, arrow.BinaryTypes.String)), - internal.ArraySpan{ + exec.ArraySpan{ Type: arrow.MapOf(arrow.PrimitiveTypes.Int8, arrow.BinaryTypes.String), Len: 1, Nulls: 1, - Buffers: [3]internal.BufferSpan{ + Buffers: [3]exec.BufferSpan{ {Buf: []byte{0}}, {Buf: []byte{0, 0, 0, 0, 0, 0, 0, 0}}, {}, }, - Children: []internal.ArraySpan{{ + Children: []exec.ArraySpan{{ Type: arrow.StructOf(arrow.Field{Name: "key", Type: arrow.PrimitiveTypes.Int8}, arrow.Field{Name: "value", Type: arrow.BinaryTypes.String, Nullable: true}), Len: 0, Nulls: 0, - Buffers: [3]internal.BufferSpan{ + Buffers: [3]exec.BufferSpan{ {Buf: []byte{}}, {}, {}, }, - Children: []internal.ArraySpan{ + Children: []exec.ArraySpan{ { Type: arrow.PrimitiveTypes.Int8, - Buffers: [3]internal.BufferSpan{ + Buffers: [3]exec.BufferSpan{ {Buf: []byte{}}, {Buf: []byte{}}, {}, }, }, { Type: arrow.BinaryTypes.String, - Buffers: [3]internal.BufferSpan{ + Buffers: [3]exec.BufferSpan{ {Buf: []byte{}}, {Buf: []byte{}}, {Buf: []byte{}}, }, }, @@ -624,21 +624,21 @@ func TestArraySpan_FillFromScalar(t *testing.T) { }}, {"list scalar", scalar.NewListScalarData(dict.Data()), - internal.ArraySpan{ + exec.ArraySpan{ Type: arrow.ListOf(arrow.BinaryTypes.String), Len: 1, Scratch: [2]uint64{ *(*uint64)(unsafe.Pointer(&[]int32{0, 2}[0])), 0, }, - Buffers: [3]internal.BufferSpan{ + Buffers: [3]exec.BufferSpan{ {Buf: []byte{0x1}}, {Buf: arrow.Int32Traits.CastToBytes([]int32{0, 2})}, }, - Children: []internal.ArraySpan{{ + Children: []exec.ArraySpan{{ Type: arrow.BinaryTypes.String, Len: 2, - Buffers: [3]internal.BufferSpan{ + Buffers: [3]exec.BufferSpan{ {Buf: dict.NullBitmapBytes(), Owner: dict.Data().Buffers()[0]}, {Buf: dict.Data().Buffers()[1].Bytes(), Owner: dict.Data().Buffers()[1]}, {Buf: dict.Data().Buffers()[2].Bytes(), Owner: dict.Data().Buffers()[2]}, @@ -648,18 +648,18 @@ func TestArraySpan_FillFromScalar(t *testing.T) { }, {"large list scalar", scalar.NewLargeListScalarData(dict.Data()), - internal.ArraySpan{ + exec.ArraySpan{ Type: arrow.LargeListOf(arrow.BinaryTypes.String), Len: 1, Scratch: [2]uint64{0, 2}, - Buffers: [3]internal.BufferSpan{ + Buffers: [3]exec.BufferSpan{ {Buf: []byte{0x1}}, {Buf: arrow.Int64Traits.CastToBytes([]int64{0, 2})}, }, - Children: []internal.ArraySpan{{ + Children: []exec.ArraySpan{{ Type: arrow.BinaryTypes.String, Len: 2, - Buffers: [3]internal.BufferSpan{ + Buffers: [3]exec.BufferSpan{ {Buf: dict.NullBitmapBytes(), Owner: dict.Data().Buffers()[0]}, {Buf: dict.Data().Buffers()[1].Bytes(), Owner: dict.Data().Buffers()[1]}, {Buf: dict.Data().Buffers()[2].Bytes(), Owner: dict.Data().Buffers()[2]}, @@ -669,17 +669,17 @@ func TestArraySpan_FillFromScalar(t *testing.T) { }, {"fixed size list", scalar.NewFixedSizeListScalar(dict), - internal.ArraySpan{ + exec.ArraySpan{ Type: arrow.FixedSizeListOf(2, arrow.BinaryTypes.String), Len: 1, - Buffers: [3]internal.BufferSpan{ + Buffers: [3]exec.BufferSpan{ {Buf: []byte{0x1}}, {}, {}, }, - Children: []internal.ArraySpan{{ + Children: []exec.ArraySpan{{ Type: arrow.BinaryTypes.String, Len: 2, - Buffers: [3]internal.BufferSpan{ + Buffers: [3]exec.BufferSpan{ {Buf: dict.NullBitmapBytes(), Owner: dict.Data().Buffers()[0]}, {Buf: dict.Data().Buffers()[1].Bytes(), Owner: dict.Data().Buffers()[1]}, {Buf: dict.Data().Buffers()[2].Bytes(), Owner: dict.Data().Buffers()[2]}, @@ -694,19 +694,19 @@ func TestArraySpan_FillFromScalar(t *testing.T) { }, []string{"int32", "uint8"}) return s }(), - internal.ArraySpan{ + exec.ArraySpan{ Type: arrow.StructOf( arrow.Field{Name: "int32", Type: arrow.PrimitiveTypes.Int32, Nullable: true}, arrow.Field{Name: "uint8", Type: arrow.PrimitiveTypes.Uint8, Nullable: true}), - Buffers: [3]internal.BufferSpan{ + Buffers: [3]exec.BufferSpan{ {Buf: []byte{0x1}}, {}, {}, }, Len: 1, - Children: []internal.ArraySpan{ + Children: []exec.ArraySpan{ { Type: arrow.PrimitiveTypes.Int32, Len: 1, - Buffers: [3]internal.BufferSpan{ + Buffers: [3]exec.BufferSpan{ {Buf: []byte{0x1}}, {Buf: arrow.Int32Traits.CastToBytes([]int32{5})}, {}, @@ -715,7 +715,7 @@ func TestArraySpan_FillFromScalar(t *testing.T) { { Type: arrow.PrimitiveTypes.Uint8, Len: 1, - Buffers: [3]internal.BufferSpan{ + Buffers: [3]exec.BufferSpan{ {Buf: []byte{0x1}}, {Buf: []byte{10}}, {}, @@ -733,7 +733,7 @@ func TestArraySpan_FillFromScalar(t *testing.T) { }, []arrow.UnionTypeCode{3, 42, 43}) return scalar.NewDenseUnionScalar(scalar.MakeScalar(uint64(25)), 42, dt.(*arrow.DenseUnionType)) }(), - internal.ArraySpan{ + exec.ArraySpan{ Type: arrow.UnionOf(arrow.DenseMode, []arrow.Field{ {Name: "string", Type: arrow.BinaryTypes.String, Nullable: true}, {Name: "number", Type: arrow.PrimitiveTypes.Uint64, Nullable: true}, @@ -741,20 +741,20 @@ func TestArraySpan_FillFromScalar(t *testing.T) { }, []arrow.UnionTypeCode{3, 42, 43}), Len: 1, Scratch: [2]uint64{42, 1}, - Buffers: [3]internal.BufferSpan{{}, + Buffers: [3]exec.BufferSpan{{}, {Buf: []byte{42}}, {Buf: arrow.Int32Traits.CastToBytes([]int32{0, 1})}, }, - Children: []internal.ArraySpan{ + Children: []exec.ArraySpan{ { Type: arrow.BinaryTypes.String, - Buffers: [3]internal.BufferSpan{ + Buffers: [3]exec.BufferSpan{ {Buf: []byte{}}, {Buf: []byte{}}, {Buf: []byte{}}, }, }, { Type: arrow.PrimitiveTypes.Uint64, Len: 1, - Buffers: [3]internal.BufferSpan{ + Buffers: [3]exec.BufferSpan{ {Buf: []byte{0x1}}, {Buf: arrow.Uint64Traits.CastToBytes([]uint64{25})}, {}, @@ -762,7 +762,7 @@ func TestArraySpan_FillFromScalar(t *testing.T) { }, { Type: arrow.PrimitiveTypes.Uint64, - Buffers: [3]internal.BufferSpan{ + Buffers: [3]exec.BufferSpan{ {Buf: []byte{}}, {Buf: []byte{}}, {}, }, }, @@ -778,7 +778,7 @@ func TestArraySpan_FillFromScalar(t *testing.T) { }, []arrow.UnionTypeCode{3, 42, 43}) return scalar.NewSparseUnionScalarFromValue(scalar.MakeScalar(uint64(25)), 1, dt.(*arrow.SparseUnionType)) }(), - internal.ArraySpan{ + exec.ArraySpan{ Type: arrow.UnionOf(arrow.SparseMode, []arrow.Field{ {Name: "string", Type: arrow.BinaryTypes.String, Nullable: true}, {Name: "number", Type: arrow.PrimitiveTypes.Uint64, Nullable: true}, @@ -786,15 +786,15 @@ func TestArraySpan_FillFromScalar(t *testing.T) { }, []arrow.UnionTypeCode{3, 42, 43}), Len: 1, Scratch: [2]uint64{42, 0}, - Buffers: [3]internal.BufferSpan{{}, + Buffers: [3]exec.BufferSpan{{}, {Buf: []byte{42}}, {}, }, - Children: []internal.ArraySpan{ + Children: []exec.ArraySpan{ { Type: arrow.BinaryTypes.String, Len: 1, Nulls: 1, - Buffers: [3]internal.BufferSpan{ + Buffers: [3]exec.BufferSpan{ {Buf: []byte{0x0}}, {Buf: []byte{0, 0, 0, 0, 0, 0, 0, 0}}, {}, @@ -803,7 +803,7 @@ func TestArraySpan_FillFromScalar(t *testing.T) { { Type: arrow.PrimitiveTypes.Uint64, Len: 1, - Buffers: [3]internal.BufferSpan{ + Buffers: [3]exec.BufferSpan{ {Buf: []byte{0x1}}, {Buf: arrow.Uint64Traits.CastToBytes([]uint64{25})}, {}, @@ -813,7 +813,7 @@ func TestArraySpan_FillFromScalar(t *testing.T) { Type: arrow.PrimitiveTypes.Uint64, Len: 1, Nulls: 1, - Buffers: [3]internal.BufferSpan{ + Buffers: [3]exec.BufferSpan{ {Buf: []byte{0x0}}, {Buf: []byte{0, 0, 0, 0, 0, 0, 0, 0}}, {}, }, }, @@ -823,9 +823,9 @@ func TestArraySpan_FillFromScalar(t *testing.T) { } for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - a := &internal.ArraySpan{ + a := &exec.ArraySpan{ Nulls: array.UnknownNullCount, - Buffers: [3]internal.BufferSpan{{SelfAlloc: true, Owner: &memory.Buffer{}}, {SelfAlloc: true, Owner: &memory.Buffer{}}, {}}, + Buffers: [3]exec.BufferSpan{{SelfAlloc: true, Owner: &memory.Buffer{}}, {SelfAlloc: true, Owner: &memory.Buffer{}}, {}}, } a.FillFromScalar(tt.args) assert.Equal(t, tt.exp, *a) diff --git a/go/arrow/compute/internal/utils.go b/go/arrow/compute/internal/exec/utils.go similarity index 97% rename from go/arrow/compute/internal/utils.go rename to go/arrow/compute/internal/exec/utils.go index 480dc33b46aa5..d95480f540fa2 100644 --- a/go/arrow/compute/internal/utils.go +++ b/go/arrow/compute/internal/exec/utils.go @@ -14,7 +14,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package internal +package exec import ( "unsafe" @@ -79,3 +79,10 @@ func GetSpanValues[T FixedWidthTypes](span *ArraySpan, i int) []T { ret := unsafe.Slice((*T)(unsafe.Pointer(&span.Buffers[i].Buf[0])), span.Offset+span.Len) return ret[span.Offset:] } + +func Min[T constraints.Ordered](a, b T) T { + if a < b { + return a + } + return b +} diff --git a/go/arrow/compute/registry_test.go b/go/arrow/compute/registry_test.go index 01f9b07949426..747a1b450b201 100644 --- a/go/arrow/compute/registry_test.go +++ b/go/arrow/compute/registry_test.go @@ -21,7 +21,9 @@ import ( "errors" "testing" + "github.com/apache/arrow/go/v10/arrow" "github.com/apache/arrow/go/v10/arrow/compute" + "github.com/apache/arrow/go/v10/arrow/compute/internal/exec" "github.com/stretchr/testify/assert" "golang.org/x/exp/slices" ) @@ -46,8 +48,10 @@ func (*mockFn) NumKernels() int { return 0 } func (*mockFn) Execute(context.Context, compute.FunctionOptions, ...compute.Datum) (compute.Datum, error) { return nil, errors.New("not implemented") } -func (*mockFn) DefaultOptions() compute.FunctionOptions { return nil } -func (*mockFn) Validate() error { return nil } +func (*mockFn) DefaultOptions() compute.FunctionOptions { return nil } +func (*mockFn) Validate() error { return nil } +func (*mockFn) DispatchExact(...arrow.DataType) (exec.Kernel, error) { return nil, nil } +func (*mockFn) DispatchBest(...arrow.DataType) (exec.Kernel, error) { return nil, nil } func TestRegistryBasics(t *testing.T) { tests := []struct { diff --git a/go/arrow/datatype.go b/go/arrow/datatype.go index 15a3c77af22c8..4a7915f9b301a 100644 --- a/go/arrow/datatype.go +++ b/go/arrow/datatype.go @@ -19,6 +19,7 @@ package arrow import ( "fmt" "hash/maphash" + "strings" "github.com/apache/arrow/go/v10/arrow/internal/debug" ) @@ -170,6 +171,21 @@ type DataType interface { Layout() DataTypeLayout } +// TypesToString is a convenience function to create a list of types +// which are comma delimited as a string +func TypesToString(types []DataType) string { + var b strings.Builder + b.WriteByte('(') + for i, t := range types { + if i != 0 { + b.WriteString(", ") + } + b.WriteString(t.String()) + } + b.WriteByte(')') + return b.String() +} + // FixedWidthDataType is the representation of an Arrow type that // requires a fixed number of bits in memory for each element. type FixedWidthDataType interface { diff --git a/go/arrow/errors.go b/go/arrow/errors.go index f8f0d48430f79..74214ae822036 100644 --- a/go/arrow/errors.go +++ b/go/arrow/errors.go @@ -19,5 +19,8 @@ package arrow import "errors" var ( - ErrInvalid = errors.New("invalid") + ErrInvalid = errors.New("invalid") + ErrNotImplemented = errors.New("not implemented") + ErrType = errors.New("type error") + ErrKey = errors.New("key error") ) diff --git a/go/arrow/internal/utils.go b/go/arrow/internal/utils.go new file mode 100644 index 0000000000000..cdaad6b91853f --- /dev/null +++ b/go/arrow/internal/utils.go @@ -0,0 +1,47 @@ +// 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 internal + +import ( + "github.com/apache/arrow/go/v10/arrow" + "github.com/apache/arrow/go/v10/arrow/internal/flatbuf" +) + +const CurMetadataVersion = flatbuf.MetadataVersionV5 + +// DefaultHasValidityBitmap is a convenience function equivalent to +// calling HasValidityBitmap with CurMetadataVersion. +func DefaultHasValidityBitmap(id arrow.Type) bool { return HasValidityBitmap(id, CurMetadataVersion) } + +// HasValidityBitmap returns whether the given type at the provided version is +// expected to have a validity bitmap in it's representation. +// +// Typically this is necessary because of the change between V4 and V5 +// where union types no longer have validity bitmaps. +func HasValidityBitmap(id arrow.Type, version flatbuf.MetadataVersion) bool { + // in <=V4 Null types had no validity bitmap + // in >=V5 Null and Union types have no validity bitmap + if version < flatbuf.MetadataVersionV5 { + return id != arrow.NULL + } + + switch id { + case arrow.NULL, arrow.DENSE_UNION, arrow.SPARSE_UNION: + return false + } + return true +} diff --git a/go/arrow/ipc/file_reader.go b/go/arrow/ipc/file_reader.go index c8fc82eca1999..8a1a9b9940d50 100644 --- a/go/arrow/ipc/file_reader.go +++ b/go/arrow/ipc/file_reader.go @@ -27,6 +27,7 @@ import ( "github.com/apache/arrow/go/v10/arrow/array" "github.com/apache/arrow/go/v10/arrow/bitutil" "github.com/apache/arrow/go/v10/arrow/endian" + "github.com/apache/arrow/go/v10/arrow/internal" "github.com/apache/arrow/go/v10/arrow/internal/dictutils" "github.com/apache/arrow/go/v10/arrow/internal/flatbuf" "github.com/apache/arrow/go/v10/arrow/memory" @@ -512,7 +513,7 @@ func (ctx *arrayLoaderContext) loadCommon(typ arrow.Type, nbufs int) (*flatbuf.F var buf *memory.Buffer - if hasValidityBitmap(typ, ctx.version) { + if internal.HasValidityBitmap(typ, flatbuf.MetadataVersion(ctx.version)) { switch field.NullCount() { case 0: ctx.ibuffer++ diff --git a/go/arrow/ipc/metadata.go b/go/arrow/ipc/metadata.go index 3a7562253658e..0bcd24df0450c 100644 --- a/go/arrow/ipc/metadata.go +++ b/go/arrow/ipc/metadata.go @@ -49,20 +49,6 @@ const ( kMaxNestingDepth = 64 ) -func hasValidityBitmap(id arrow.Type, version MetadataVersion) bool { - // in <=V4 Null types had no validity bitmap - // in >=V5 Null and Union types have no validity bitmap - if version < MetadataV5 { - return id != arrow.NULL - } - - switch id { - case arrow.NULL, arrow.DENSE_UNION, arrow.SPARSE_UNION: - return false - } - return true -} - type startVecFunc func(b *flatbuffers.Builder, n int) flatbuffers.UOffsetT type fieldMetadata struct { diff --git a/go/arrow/ipc/writer.go b/go/arrow/ipc/writer.go index 0cfa9e7ee2b20..9af88d9c2d78d 100644 --- a/go/arrow/ipc/writer.go +++ b/go/arrow/ipc/writer.go @@ -30,6 +30,7 @@ import ( "github.com/apache/arrow/go/v10/arrow" "github.com/apache/arrow/go/v10/arrow/array" "github.com/apache/arrow/go/v10/arrow/bitutil" + "github.com/apache/arrow/go/v10/arrow/internal" "github.com/apache/arrow/go/v10/arrow/internal/debug" "github.com/apache/arrow/go/v10/arrow/internal/dictutils" "github.com/apache/arrow/go/v10/arrow/internal/flatbuf" @@ -479,7 +480,7 @@ func (w *recordEncoder) visit(p *Payload, arr arrow.Array) error { return nil } - if hasValidityBitmap(arr.DataType().ID(), currentMetadataVersion) { + if internal.HasValidityBitmap(arr.DataType().ID(), flatbuf.MetadataVersion(currentMetadataVersion)) { switch arr.NullN() { case 0: // there are no null values, drop the null bitmap diff --git a/go/arrow/memory/buffer.go b/go/arrow/memory/buffer.go index 87efddeebd364..cbffc7c23f009 100644 --- a/go/arrow/memory/buffer.go +++ b/go/arrow/memory/buffer.go @@ -48,6 +48,10 @@ func SliceBuffer(buf *Buffer, offset, length int) *Buffer { return &Buffer{refCount: 1, parent: buf, buf: buf.Bytes()[offset : offset+length], length: length} } +// Parent returns either nil or a pointer to the parent buffer if this buffer +// was sliced from another. +func (b *Buffer) Parent() *Buffer { return b.parent } + // Retain increases the reference count by 1. func (b *Buffer) Retain() { if b.mem != nil || b.parent != nil {