Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

ARROW-17455: [Go] Function and Kernel execution architecture #13964

Merged
merged 6 commits into from
Aug 25, 2022
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
23 changes: 23 additions & 0 deletions go.work
Original file line number Diff line number Diff line change
@@ -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
)
43 changes: 43 additions & 0 deletions go/arrow/bitutil/bitmaps.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@
package bitutil

import (
"bytes"
"math/bits"
"unsafe"

Expand Down Expand Up @@ -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
}
27 changes: 20 additions & 7 deletions go/arrow/compute/datum.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down Expand Up @@ -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}
Expand Down
5 changes: 2 additions & 3 deletions go/arrow/compute/datumkind_string.go

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

165 changes: 165 additions & 0 deletions go/arrow/compute/exec.go
Original file line number Diff line number Diff line change
@@ -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...)
}
Loading