-
Notifications
You must be signed in to change notification settings - Fork 4.3k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[Go SDK] Dataframe API wrapper (#23450)
- Loading branch information
1 parent
107a43d
commit 78e1c0a
Showing
7 changed files
with
343 additions
and
0 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,87 @@ | ||
// 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 dataframe is a wrapper for DataframeTransform defined in Apache Beam Python SDK. | ||
// An exapnsion service for python external transforms can be started by running | ||
// $ python -m apache_beam.runners.portability.expansion_service_main -p $PORT_FOR_EXPANSION_SERVICE | ||
package dataframe | ||
|
||
import ( | ||
"reflect" | ||
|
||
"github.com/apache/beam/sdks/v2/go/pkg/beam" | ||
"github.com/apache/beam/sdks/v2/go/pkg/beam/core/typex" | ||
"github.com/apache/beam/sdks/v2/go/pkg/beam/transforms/xlang/python" | ||
) | ||
|
||
func init() { | ||
beam.RegisterType(reflect.TypeOf((*config)(nil)).Elem()) | ||
beam.RegisterType(reflect.TypeOf((*kwargs)(nil)).Elem()) | ||
beam.RegisterType(reflect.TypeOf((*argStruct)(nil)).Elem()) | ||
} | ||
|
||
type kwargs struct { | ||
Fn python.CallableSource `beam:"func"` | ||
IncludeIndexes bool `beam:"include_indexes"` | ||
} | ||
|
||
type argStruct struct{} | ||
|
||
type config struct { | ||
dpl kwargs | ||
expansionAddr string | ||
} | ||
|
||
type configOption func(*config) | ||
|
||
// WithExpansionAddr sets an URL for a Python expansion service. | ||
func WithExpansionAddr(expansionAddr string) configOption { | ||
return func(c *config) { | ||
c.expansionAddr = expansionAddr | ||
} | ||
} | ||
|
||
// WithIndexes sets include_indexes option for DataframeTransform. | ||
func WithIndexes() configOption { | ||
return func(c *config) { | ||
c.dpl.IncludeIndexes = true | ||
} | ||
} | ||
|
||
// Transform is a multi-language wrapper for a Python DataframeTransform with a given lambda function. | ||
// lambda function is a required parameter. | ||
// Additional option for including indexes in dataframe can be provided by using | ||
// dataframe.WithIndexes(). | ||
func Transform(s beam.Scope, fn string, col beam.PCollection, outT reflect.Type, opts ...configOption) beam.PCollection { | ||
s.Scope("xlang.python.DataframeTransform") | ||
cfg := config{ | ||
dpl: kwargs{Fn: python.CallableSource(fn)}, | ||
} | ||
for _, opt := range opts { | ||
opt(&cfg) | ||
} | ||
|
||
// TODO: load automatic expansion service here | ||
if cfg.expansionAddr == "" { | ||
panic("no expansion service address provided for xlang.DataframeTransform(), pass xlang.WithExpansionAddr(address) as a param.") | ||
} | ||
|
||
pet := python.NewExternalTransform[argStruct, kwargs]("apache_beam.dataframe.transforms.DataframeTransform") | ||
pet.WithKwargs(cfg.dpl) | ||
pl := beam.CrossLanguagePayload(pet) | ||
result := beam.CrossLanguage(s, "beam:transforms:python:fully_qualified_named", pl, cfg.expansionAddr, beam.UnnamedInput(col), beam.UnnamedOutput(typex.New(outT))) | ||
return result[beam.UnnamedOutputTag()] | ||
|
||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,129 @@ | ||
// 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 python contains data structures required for python external transforms in a multilanguage pipeline. | ||
package python | ||
|
||
import ( | ||
"fmt" | ||
"io" | ||
"reflect" | ||
|
||
"github.com/apache/beam/sdks/v2/go/pkg/beam" | ||
"github.com/apache/beam/sdks/v2/go/pkg/beam/core/graph/coder" | ||
"github.com/apache/beam/sdks/v2/go/pkg/beam/core/util/reflectx" | ||
) | ||
|
||
const ( | ||
pythonCallableUrn = "beam:logical_type:python_callable:v1" | ||
) | ||
|
||
var ( | ||
pcsType = reflect.TypeOf((*CallableSource)(nil)).Elem() | ||
pcsStorageType = reflectx.String | ||
) | ||
|
||
func init() { | ||
beam.RegisterType(pcsType) | ||
beam.RegisterSchemaProviderWithURN(pcsType, &callableSourceProvider{}, pythonCallableUrn) | ||
} | ||
|
||
// CallableSource is a wrapper object storing a Python function definition | ||
// that can be evaluated to Python callables in Python SDK. | ||
// | ||
// The snippet of Python code can be a valid Python expression such as | ||
// lambda x: x * x | ||
// str.upper | ||
// a fully qualified name such as | ||
// math.sin | ||
// or a complete multi-line function or class definition such as | ||
// def foo(x): | ||
// ... | ||
// class Foo: | ||
// ... | ||
// | ||
// Any lines preceding the function definition are first evaluated to provide context in which to | ||
// define the function which can be useful to declare imports or any other needed values, e.g. | ||
// import math | ||
// | ||
// def helper(x): | ||
// return x * x | ||
// | ||
// def func(y): | ||
// return helper(y) + y | ||
// in which case `func` would get applied to each element. | ||
type CallableSource string | ||
|
||
// callableSourceProvider implement the SchemaProvider interface for logical types | ||
type callableSourceProvider struct{} | ||
|
||
// FromLogicalType returns the goType of the logical type | ||
func (p *callableSourceProvider) FromLogicalType(rt reflect.Type) (reflect.Type, error) { | ||
if rt != pcsType { | ||
return nil, fmt.Errorf("unable to provide schema.LogicalType for type %v, want %v", rt, pcsType) | ||
} | ||
return pcsStorageType, nil | ||
} | ||
|
||
// BuildEncoder encodes the PythonCallableSource logical type | ||
func (p *callableSourceProvider) BuildEncoder(rt reflect.Type) (func(interface{}, io.Writer) error, error) { | ||
if _, err := p.FromLogicalType(rt); err != nil { | ||
return nil, err | ||
} | ||
|
||
return func(iface interface{}, w io.Writer) error { | ||
v := iface.(CallableSource) | ||
return coder.EncodeStringUTF8(string(v), w) | ||
}, nil | ||
} | ||
|
||
// BuildDecoder decodes the PythonCallableSource logical type | ||
func (p *callableSourceProvider) BuildDecoder(rt reflect.Type) (func(io.Reader) (interface{}, error), error) { | ||
if _, err := p.FromLogicalType(rt); err != nil { | ||
return nil, err | ||
} | ||
|
||
return func(r io.Reader) (interface{}, error) { | ||
s, err := coder.DecodeStringUTF8(r) | ||
if err != nil { | ||
return nil, err | ||
} | ||
return CallableSource(s), nil | ||
}, nil | ||
} | ||
|
||
// NewExternalTransform creates a new instance for python external transform. It accepts two types: | ||
// A: used for normal arguments | ||
// K: used for keyword arguments | ||
func NewExternalTransform[A, K any](constructor string) *pythonExternalTransform[A, K] { | ||
return &pythonExternalTransform[A, K]{Constructor: constructor} | ||
} | ||
|
||
// PythonExternalTransform holds the details required for an External Python Transform. | ||
type pythonExternalTransform[A, K any] struct { | ||
Constructor string `beam:"constructor"` | ||
Args A `beam:"args"` | ||
Kwargs K `beam:"kwargs"` | ||
} | ||
|
||
// WithArgs adds arguments to the External Python Transform. | ||
func (p *pythonExternalTransform[A, K]) WithArgs(args any) { | ||
p.Args = args.(A) | ||
} | ||
|
||
// WithKwargs adds keyword arguments to the External Python Transform. | ||
func (p *pythonExternalTransform[A, K]) WithKwargs(kwargs any) { | ||
p.Kwargs = kwargs.(K) | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
48 changes: 48 additions & 0 deletions
48
sdks/go/test/integration/transforms/xlang/dataframe/dataframe.go
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,48 @@ | ||
// 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 dataframe | ||
|
||
import ( | ||
"reflect" | ||
|
||
"github.com/apache/beam/sdks/v2/go/pkg/beam" | ||
"github.com/apache/beam/sdks/v2/go/pkg/beam/testing/passert" | ||
"github.com/apache/beam/sdks/v2/go/pkg/beam/transforms/xlang/dataframe" | ||
) | ||
|
||
func init() { | ||
beam.RegisterType(reflect.TypeOf((*TestRow)(nil)).Elem()) | ||
} | ||
|
||
type TestRow struct { | ||
A int64 `beam:"a"` | ||
B int32 `beam:"b"` | ||
} | ||
|
||
func DataframeTransform(expansionAddr string) *beam.Pipeline { | ||
row0 := TestRow{A: int64(100), B: int32(1)} | ||
row1 := TestRow{A: int64(100), B: int32(2)} | ||
row2 := TestRow{A: int64(100), B: int32(3)} | ||
row3 := TestRow{A: int64(200), B: int32(4)} | ||
|
||
p, s := beam.NewPipelineWithRoot() | ||
|
||
input := beam.Create(s, row0, row1, row3) | ||
outCol := dataframe.Transform(s, "lambda df: df.groupby('a').sum()", input, reflect.TypeOf((*TestRow)(nil)).Elem(), dataframe.WithExpansionAddr(expansionAddr), dataframe.WithIndexes()) | ||
|
||
passert.Equals(s, outCol, row2, row3) | ||
return p | ||
} |
60 changes: 60 additions & 0 deletions
60
sdks/go/test/integration/transforms/xlang/dataframe/dataframe_test.go
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,60 @@ | ||
// 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 dataframe | ||
|
||
import ( | ||
"flag" | ||
"log" | ||
"testing" | ||
|
||
"github.com/apache/beam/sdks/v2/go/pkg/beam" | ||
_ "github.com/apache/beam/sdks/v2/go/pkg/beam/runners/dataflow" | ||
_ "github.com/apache/beam/sdks/v2/go/pkg/beam/runners/flink" | ||
_ "github.com/apache/beam/sdks/v2/go/pkg/beam/runners/universal" | ||
"github.com/apache/beam/sdks/v2/go/pkg/beam/testing/ptest" | ||
"github.com/apache/beam/sdks/v2/go/test/integration" | ||
) | ||
|
||
var expansionAddr string // Populate with expansion address labelled "python_transform". | ||
|
||
func checkFlags(t *testing.T) { | ||
if expansionAddr == "" { | ||
t.Skip("No python transform expansion address provided.") | ||
} | ||
} | ||
|
||
func TestDataframe(t *testing.T) { | ||
integration.CheckFilters(t) | ||
checkFlags(t) | ||
p := DataframeTransform(expansionAddr) | ||
ptest.RunAndValidate(t, p) | ||
} | ||
|
||
func TestMain(m *testing.M) { | ||
flag.Parse() | ||
beam.Init() | ||
|
||
services := integration.NewExpansionServices() | ||
defer func() { services.Shutdown() }() | ||
addr, err := services.GetAddr("python_transform") | ||
if err != nil { | ||
log.Printf("skipping missing expansion service: %v", err) | ||
} else { | ||
expansionAddr = addr | ||
} | ||
|
||
ptest.MainRet(m) | ||
} |