Documentation ¶
Overview ¶
Package beam is an implementation of the Apache Beam (https://beam.apache.org) programming model in Go. Beam provides a simple, powerful model for building both batch and streaming parallel data processing pipelines.
For more on the Beam model see: https://beam.apache.org/documentation/programming-guide
For design choices this implementation makes see: https://s.apache.org/beam-go-sdk-design-rfc
Example (GettingStarted) ¶
package main import ( "context" "fmt" "regexp" "strings" "github.com/apache/beam/sdks/go/pkg/beam" "github.com/apache/beam/sdks/go/pkg/beam/io/textio" "github.com/apache/beam/sdks/go/pkg/beam/runners/direct" ) func main() { // In order to start creating the pipeline for execution, a Pipeline object is needed. p := beam.NewPipeline() s := p.Root() // The pipeline object encapsulates all the data and steps in your processing task. // It is the basis for creating the pipeline's data sets as PCollections and its operations // as transforms. // The PCollection abstraction represents a potentially distributed, // multi-element data set. You can think of a PCollection as “pipeline” data; // Beam transforms use PCollection objects as inputs and outputs. As such, if // you want to work with data in your pipeline, it must be in the form of a // PCollection. // Transformations are applied in a scoped fashion to the pipeline. The scope // can be obtained from the pipeline object. // Start by reading text from an input files, and receiving a PCollection. lines := textio.Read(s, "protocol://path/file*.txt") // Transforms are added to the pipeline so they are part of the work to be // executed. Since this transform has no PCollection as an input, it is // considered a 'root transform' // A pipeline can have multiple root transforms moreLines := textio.Read(s, "protocol://other/path/file*.txt") // Further transforms can be applied, creating an arbitrary, acyclic graph. // Subsequent transforms (and the intermediate PCollections they produce) are // attached to the same pipeline. all := beam.Flatten(s, lines, moreLines) wordRegexp := regexp.MustCompile(`[a-zA-Z]+('[a-z])?`) words := beam.ParDo(s, func(line string, emit func(string)) { for _, word := range wordRegexp.FindAllString(line, -1) { emit(word) } }, all) formatted := beam.ParDo(s, strings.ToUpper, words) textio.Write(s, "protocol://output/path", formatted) // Applying a transform adds it to the pipeline, rather than executing it // immediately. Once the whole pipeline of transforms is constructed, the // pipeline can be executed by a PipelineRunner. The direct runner executes the // transforms directly, sequentially, in this one process, which is useful for // unit tests and simple experiments: if _, err := direct.Execute(context.Background(), p); err != nil { fmt.Printf("Pipeline failed: %v", err) } }
Output:
Example (MetricsDeclaredAnywhere) ¶
package main import ( "context" "regexp" "github.com/apache/beam/sdks/go/pkg/beam" "github.com/apache/beam/sdks/go/pkg/beam/core/metrics" ) func ctxWithPtransformID(id string) context.Context { ctx := context.Background() ctx = metrics.SetBundleID(ctx, "exampleBundle") ctx = metrics.SetPTransformID(ctx, id) return ctx } var wordRE = regexp.MustCompile(`[a-zA-Z]+('[a-z])?`) func main() { // Metrics can be declared outside DoFns, and used inside.. outside := beam.NewCounter("example.namespace", "count") extractWordsDofn := func(ctx context.Context, line string, emit func(string)) { // They can be defined at time of use within a DoFn, if necessary. inside := beam.NewDistribution("example.namespace", "characters") for _, word := range wordRE.FindAllString(line, -1) { emit(word) outside.Inc(ctx, 1) inside.Update(ctx, int64(len(word))) } } ctx := ctxWithPtransformID("example") extractWordsDofn(ctx, "this has six words in it", func(string) {}) extractWordsDofn(ctx, "this has seven words in it, see?", func(string) {}) metrics.DumpToOutFromContext(ctx) }
Output: PTransformID: "example" example.namespace.characters - count: 13 sum: 43 min: 2 max: 5 example.namespace.count - value: 13
Example (MetricsReusable) ¶
package main import ( "context" "regexp" "github.com/apache/beam/sdks/go/pkg/beam" "github.com/apache/beam/sdks/go/pkg/beam/core/metrics" ) // A beam_test global context var to improve how the examples look. var ctx = context.Background() var wordRE = regexp.MustCompile(`[a-zA-Z]+('[a-z])?`) func main() { // Metric proxies can be used in multiple DoFns c := beam.NewCounter("example.reusable", "count") extractWordsDofn := func(ctx context.Context, line string, emit func(string)) { for _, word := range wordRE.FindAllString(line, -1) { emit(word) c.Inc(ctx, 1) } } extractRunesDofn := func(ctx context.Context, line string, emit func(rune)) { for _, r := range line { emit(r) c.Inc(ctx, 1) } } ctx = metrics.SetBundleID(ctx, "exampleBundle") extractWordsDofn(metrics.SetPTransformID(ctx, "extract1"), "this has six words in it", func(string) {}) extractRunesDofn(metrics.SetPTransformID(ctx, "extract2"), "seven thousand", func(rune) {}) metrics.DumpToOutFromContext(ctx) }
Output: PTransformID: "extract1" example.reusable.count - value: 6 PTransformID: "extract2" example.reusable.count - value: 14
Index ¶
- Variables
- func CrossLanguage(s Scope, urn string, payload []byte, expansionAddr string, ...) map[string]PCollection
- func Init()
- func Initialized() bool
- func NewPipelineWithRoot() (*Pipeline, Scope)
- func ParDo0(s Scope, dofn interface{}, col PCollection, opts ...Option)
- func ParDo2(s Scope, dofn interface{}, col PCollection, opts ...Option) (PCollection, PCollection)
- func ParDo3(s Scope, dofn interface{}, col PCollection, opts ...Option) (PCollection, PCollection, PCollection)
- func ParDo4(s Scope, dofn interface{}, col PCollection, opts ...Option) (PCollection, PCollection, PCollection, PCollection)
- func ParDo5(s Scope, dofn interface{}, col PCollection, opts ...Option) (PCollection, PCollection, PCollection, PCollection, PCollection)
- func ParDo6(s Scope, dofn interface{}, col PCollection, opts ...Option) (PCollection, PCollection, PCollection, PCollection, PCollection, PCollection)
- func ParDo7(s Scope, dofn interface{}, col PCollection, opts ...Option) (PCollection, PCollection, PCollection, PCollection, PCollection, PCollection, ...)
- func RegisterCoder(t reflect.Type, encoder, decoder interface{})
- func RegisterDoFn(dofn interface{})
- func RegisterFunction(fn interface{})
- func RegisterInit(hook func())
- func RegisterRunner(name string, fn func(ctx context.Context, p *Pipeline) (PipelineResult, error))
- func RegisterSchemaProvider(rt reflect.Type, provider interface{})
- func RegisterType(t reflect.Type)
- func TryCrossLanguage(s Scope, ext *graph.ExternalTransform, ins []*graph.Inbound, ...) (map[string]*graph.Node, error)
- func UnnamedInput(col PCollection) map[string]PCollection
- func UnnamedOutput(t FullType) map[string]FullType
- func ValidateKVType(col PCollection) (typex.FullType, typex.FullType)
- func ValidateNonCompositeType(col PCollection) typex.FullType
- type Coder
- type Counter
- type Distribution
- type ElementDecoder
- type ElementEncoder
- type EncodedCoder
- type EncodedFunc
- type EncodedType
- type EventTime
- type FullType
- type Gauge
- type Option
- type PCollection
- func AddFixedKey(s Scope, col PCollection) PCollection
- func CoGroupByKey(s Scope, cols ...PCollection) PCollection
- func Combine(s Scope, combinefn interface{}, col PCollection, opts ...Option) PCollection
- func CombinePerKey(s Scope, combinefn interface{}, col PCollection, opts ...Option) PCollection
- func Create(s Scope, values ...interface{}) PCollection
- func CreateList(s Scope, list interface{}) PCollection
- func DropKey(s Scope, col PCollection) PCollection
- func DropValue(s Scope, col PCollection) PCollection
- func Explode(s Scope, col PCollection) PCollection
- func External(s Scope, spec string, payload []byte, in []PCollection, out []FullType, ...) []PCollection
- func Flatten(s Scope, cols ...PCollection) PCollection
- func GroupByKey(s Scope, a PCollection) PCollection
- func Impulse(s Scope) PCollection
- func ImpulseValue(s Scope, value []byte) PCollection
- func Must(a PCollection, err error) PCollection
- func MustN(list []PCollection, err error) []PCollection
- func ParDo(s Scope, dofn interface{}, col PCollection, opts ...Option) PCollection
- func ParDoN(s Scope, dofn interface{}, col PCollection, opts ...Option) []PCollection
- func Partition(s Scope, n int, fn interface{}, col PCollection) []PCollection
- func Reshuffle(s Scope, col PCollection) PCollection
- func Seq(s Scope, col PCollection, dofns ...interface{}) PCollection
- func SwapKV(s Scope, col PCollection) PCollection
- func TryCoGroupByKey(s Scope, cols ...PCollection) (PCollection, error)
- func TryCombine(s Scope, combinefn interface{}, col PCollection, opts ...Option) (PCollection, error)
- func TryCombinePerKey(s Scope, combinefn interface{}, col PCollection, opts ...Option) (PCollection, error)
- func TryCreate(s Scope, values ...interface{}) (PCollection, error)
- func TryCreateList(s Scope, list interface{}) (PCollection, error)
- func TryExternal(s Scope, spec string, payload []byte, in []PCollection, out []FullType, ...) ([]PCollection, error)
- func TryFlatten(s Scope, cols ...PCollection) (PCollection, error)
- func TryGroupByKey(s Scope, a PCollection) (PCollection, error)
- func TryParDo(s Scope, dofn interface{}, col PCollection, opts ...Option) ([]PCollection, error)
- func TryReshuffle(s Scope, col PCollection) (PCollection, error)
- func TryWindowInto(s Scope, ws *window.Fn, col PCollection) (PCollection, error)
- func WindowInto(s Scope, ws *window.Fn, col PCollection) PCollection
- type Pipeline
- type PipelineResult
- type SchemaProvider
- type Scope
- type SideInput
- type T
- type TypeDefinition
- type U
- type V
- type W
- type Window
- type X
- type Y
- type Z
Examples ¶
Constants ¶
This section is empty.
Variables ¶
var ( TType = typex.TType UType = typex.UType VType = typex.VType WType = typex.WType XType = typex.XType YType = typex.YType ZType = typex.ZType )
These are the reflect.Type instances of the universal types, which are used when binding actual types to "generic" DoFns that use Universal Types.
var EnableSchemas bool = false
EnableSchemas is a temporary configuration variable to use Beam Schema encoding by default instead of JSON. Before it is removed, it will be set to true by default and then eventually removed.
Only users who rely on default JSON marshalling behaviour should set this explicitly.
var EventTimeType = typex.EventTimeType
EventTimeType is the reflect.Type of EventTime.
var PipelineOptions = runtime.GlobalOptions
PipelineOptions are global options for the active pipeline. Options can be defined any time before execution and are re-created by the harness on remote execution workers. Global options should be used sparingly.
Functions ¶
func CrossLanguage ¶
func CrossLanguage( s Scope, urn string, payload []byte, expansionAddr string, namedInputs map[string]PCollection, namedOutputTypes map[string]FullType, ) map[string]PCollection
CrossLanguage executes a cross-language transform that uses named inputs and returns named outputs.
func Init ¶
func Init()
Init is the hook that all user code must call after flags processing and other static initialization, for now.
func Initialized ¶
func Initialized() bool
Initialized exposes the initialization status for runners.
func NewPipelineWithRoot ¶
NewPipelineWithRoot creates a new empty pipeline and its root scope.
func ParDo0 ¶
func ParDo0(s Scope, dofn interface{}, col PCollection, opts ...Option)
ParDo0 inserts a ParDo with zero output transform into the pipeline.
func ParDo2 ¶
func ParDo2(s Scope, dofn interface{}, col PCollection, opts ...Option) (PCollection, PCollection)
ParDo2 inserts a ParDo with 2 outputs into the pipeline.
func ParDo3 ¶
func ParDo3(s Scope, dofn interface{}, col PCollection, opts ...Option) (PCollection, PCollection, PCollection)
ParDo3 inserts a ParDo with 3 outputs into the pipeline.
func ParDo4 ¶
func ParDo4(s Scope, dofn interface{}, col PCollection, opts ...Option) (PCollection, PCollection, PCollection, PCollection)
ParDo4 inserts a ParDo with 4 outputs into the pipeline.
func ParDo5 ¶
func ParDo5(s Scope, dofn interface{}, col PCollection, opts ...Option) (PCollection, PCollection, PCollection, PCollection, PCollection)
ParDo5 inserts a ParDo with 5 outputs into the pipeline.
func ParDo6 ¶
func ParDo6(s Scope, dofn interface{}, col PCollection, opts ...Option) (PCollection, PCollection, PCollection, PCollection, PCollection, PCollection)
ParDo6 inserts a ParDo with 6 outputs into the pipeline.
func ParDo7 ¶
func ParDo7(s Scope, dofn interface{}, col PCollection, opts ...Option) (PCollection, PCollection, PCollection, PCollection, PCollection, PCollection, PCollection)
ParDo7 inserts a ParDo with 7 outputs into the pipeline.
func RegisterCoder ¶
RegisterCoder registers a user defined coder for a given type, and will be used if there is no existing beam coder for that type. Must be called prior to beam.Init(), preferably in an init() function.
The coder used for a given type follows this ordering:
- Coders for Known Beam types.
- Coders registered for specific types
- Coders registered for interfaces types
- Default coder (JSON)
Coders for interface types are iterated over to check if a type satisfies them, and the most recent one registered will be used.
Repeated registrations of the same type overrides prior ones.
RegisterCoder additionally registers the type, and coder functions as per RegisterType and RegisterFunction to avoid redundant calls.
Supported Encoder Signatures
func(T) []byte func(reflect.Type, T) []byte func(T) ([]byte, error) func(reflect.Type, T) ([]byte, error)
Supported Decoder Signatures
func([]byte) T func(reflect.Type, []byte) T func([]byte) (T, error) func(reflect.Type, []byte) (T, error)
where T is the matching user type.
func RegisterDoFn ¶
func RegisterDoFn(dofn interface{})
RegisterDoFn is a convenience function to handle registering a DoFn and all related types. Use this instead of calling RegisterType or RegisterFunction. Like all the Register* functions, RegisterDoFn should be called in `init()` only.
In particular, it will call RegisterFunction for functional DoFns, and RegisterType for the parameter and return types for that function. StructuralDoFns will have RegisterType called for itself and the parameter and return types.
RegisterDoFn will panic if the argument type is not a DoFn.
Usage:
func init() { beam.RegisterDoFn(FunctionalDoFn) beam.RegisterDoFn(reflect.TypeOf((*StructuralDoFn)(nil)).Elem()) }
func RegisterFunction ¶
func RegisterFunction(fn interface{})
RegisterFunction allows function registration. It is beneficial for performance and is needed for functions -- such as custom coders -- serialized during unit tests, where the underlying symbol table is not available. It should be called in `init()` only.
func RegisterInit ¶
func RegisterInit(hook func())
RegisterInit registers an Init hook. Hooks are expected to be able to figure out whether they apply on their own, notably if invoked in a remote execution environment. They are all executed regardless of the runner.
func RegisterRunner ¶
RegisterRunner associates the name with the supplied runner, making it available to execute a pipeline via Run.
func RegisterSchemaProvider ¶
RegisterSchemaProvider allows pipeline authors to provide special handling to convert types to schema representations, when those types are used as fields in types being encoded as schema rows.
At present, the only supported provider interface is SchemaProvider, though this may change in the future.
Providers only need to support a limited set of types for conversion, specifically a single struct type or a pointer to struct type, or an interface type, which they are registered with.
Providers have three tasks with respect to a given supported logical type:
- Producing schema representative types for their logical types.
- Producing schema encoders for values of that type, writing beam schema encoded bytes for a value, matching the schema representative type.
- Producing schema decoders for values of that type, reading beam schema encoded bytes, and producing a value of that type.
Representative Schema types must be structs with only exported fields.
A provider should be thread safe, but it's not required that a produced encoder or decoder is thread safe, since a separate encoder or decoder will be used for simultaneously executed bundles.
If the supported type is an interface, that interface must have a non-empty method set. That is, it cannot be the empty interface.
RegisterSchemaProvider must be called before beam.Init(), and conventionally is called in a package init() function.
Example ¶
// 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 main import ( "bytes" "fmt" "io" "reflect" "github.com/apache/beam/sdks/go/pkg/beam" "github.com/apache/beam/sdks/go/pkg/beam/core/graph/coder" "github.com/google/go-cmp/cmp" ) // RegisterSchemaProvider must be called before beam.Init, and conventionally in a package init block. func init() { beam.RegisterSchemaProvider(reflect.TypeOf((*Alphabet)(nil)).Elem(), &AlphabetProvider{}) // TODO(BEAM-9615): Registerying a self encoding type causes a cycle. Needs resolving. // beam.RegisterType(reflect.TypeOf((*Cyrillic)(nil))) beam.RegisterType(reflect.TypeOf((*Latin)(nil))) beam.RegisterType(reflect.TypeOf((*Ελληνικά)(nil))) } type Alphabet interface { alphabet() string } type Cyrillic struct { A, B int } func (*Cyrillic) alphabet() string { return "Cyrillic" } type Latin struct { // Unexported fields are not serializable by beam schemas by default // so we need to handle this ourselves. c uint64 d *float32 } func (*Latin) alphabet() string { return "Latin" } type Ελληνικά struct { q string G func() string } func (*Ελληνικά) alphabet() string { return "Ελληνικά" } // AlphabetProvider provides encodings for types that implement the Alphabet interface. type AlphabetProvider struct { enc *coder.RowEncoderBuilder dec *coder.RowDecoderBuilder } var ( typeCyrillic = reflect.TypeOf((*Cyrillic)(nil)) typeLatin = reflect.TypeOf((*Latin)(nil)) typeΕλληνικά = reflect.TypeOf((*Ελληνικά)(nil)) ) func (p *AlphabetProvider) FromLogicalType(rt reflect.Type) (reflect.Type, error) { // FromLogicalType produces schema representative types, which match the encoders // and decoders that this function generates for this type. // While this example uses statically assigned schema representative types, it's // possible to generate the returned reflect.Type dynamically instead, using the // reflect package. switch rt { // The Cyrillic type is able to be encoded by default, so we simply use it directly // as it's own representative type. case typeCyrillic: return typeCyrillic, nil case typeLatin: // The Latin type only has unexported fields, so we need to make the equivalent // have exported fields. return reflect.TypeOf((*struct { C uint64 D *float32 })(nil)).Elem(), nil case typeΕλληνικά: return reflect.TypeOf((*struct{ Q string })(nil)).Elem(), nil } return nil, fmt.Errorf("Unknown Alphabet: %v", rt) } // BuildEncoder returns beam schema encoder functions for types with the Alphabet interface. func (p *AlphabetProvider) BuildEncoder(rt reflect.Type) (func(interface{}, io.Writer) error, error) { switch rt { case typeCyrillic: if p.enc == nil { p.enc = &coder.RowEncoderBuilder{} } // Since Cyrillic is by default encodable, defer to the standard schema row decoder for the type. return p.enc.Build(rt) case typeLatin: return func(iface interface{}, w io.Writer) error { v := iface.(*Latin) // Beam Schema Rows have a header that indicates which fields if any, are nil. if err := coder.WriteRowHeader(2, func(i int) bool { if i == 1 { return v.d == nil } return false }, w); err != nil { return err } // Afterwards, each field is encoded using the appropriate helper. if err := coder.EncodeVarUint64(v.c, w); err != nil { return err } // Nil fields have nothing written for them other than the header. if v.d != nil { if err := coder.EncodeDouble(float64(*v.d), w); err != nil { return err } } return nil }, nil case typeΕλληνικά: return func(iface interface{}, w io.Writer) error { // Since the representation for Ελληνικά never has nil fields // we can use the simple header helper. if err := coder.WriteSimpleRowHeader(1, w); err != nil { return err } v := iface.(*Ελληνικά) if err := coder.EncodeStringUTF8(v.q, w); err != nil { return fmt.Errorf("decoding string field A: %v", err) } return nil }, nil } return nil, fmt.Errorf("Unknown Alphabet: %v", rt) } // BuildDecoder returns beam schema decoder functions for types with the Alphabet interface. func (p *AlphabetProvider) BuildDecoder(rt reflect.Type) (func(io.Reader) (interface{}, error), error) { switch rt { case typeCyrillic: if p.dec == nil { p.dec = &coder.RowDecoderBuilder{} } // Since Cyrillic is by default encodable, defer to the standard schema row decoder for the type. return p.dec.Build(rt) case typeLatin: return func(r io.Reader) (interface{}, error) { // Since the d field can be nil, we use the header get the nil bits. n, nils, err := coder.ReadRowHeader(r) if err != nil { return nil, err } // Header returns the number of fields, so we check if it has what we // expect. This allows schemas to evolve if necessary. if n != 2 { return nil, fmt.Errorf("expected 2 fields, but got %v", n) } c, err := coder.DecodeVarUint64(r) if err != nil { return nil, err } // Check if the field is nil before trying to decode a value for it. var d *float32 if !coder.IsFieldNil(nils, 1) { f, err := coder.DecodeDouble(r) if err != nil { return nil, err } f32 := float32(f) d = &f32 } return &Latin{ c: c, d: d, }, nil }, nil case typeΕλληνικά: return func(r io.Reader) (interface{}, error) { // Since the representation for Ελληνικά never has nil fields // we can use the simple header helper. Returns an error if // something unexpected occurs. if err := coder.ReadSimpleRowHeader(1, r); err != nil { return nil, err } q, err := coder.DecodeStringUTF8(r) if err != nil { return nil, fmt.Errorf("decoding string field A: %v", err) } return &Ελληνικά{ q: q, }, nil }, nil } return nil, nil } // Schema providers work on fields of schema encoded types. type translation struct { C *Cyrillic L *Latin E *Ελληνικά } func main() { f := float32(42.789) want := translation{ C: &Cyrillic{A: 123, B: 456}, L: &Latin{c: 789, d: &f}, E: &Ελληνικά{q: "testing"}, } rt := reflect.TypeOf((*translation)(nil)).Elem() enc, err := coder.RowEncoderForStruct(rt) if err != nil { panic(err) } dec, err := coder.RowDecoderForStruct(rt) if err != nil { panic(err) } var buf bytes.Buffer if err := enc(want, &buf); err != nil { panic(err) } got, err := dec(&buf) if err != nil { panic(err) } if d := cmp.Diff(want, got, cmp.AllowUnexported(Latin{}, Ελληνικά{})); d != "" { fmt.Printf("diff in schema encoding translation: (-want,+got)\n%v\n", d) } else { fmt.Println("No diffs!") } }
Output: No diffs!
func RegisterType ¶
RegisterType inserts "external" types into a global type registry to bypass serialization and preserve full method information. It should be called in `init()` only. TODO(wcn): the canonical definition of "external" is in v1.proto. We need user facing copy for this important concept.
func TryCrossLanguage ¶
func TryCrossLanguage(s Scope, ext *graph.ExternalTransform, ins []*graph.Inbound, outs []*graph.Outbound) (map[string]*graph.Node, error)
TryCrossLanguage coordinates the core functions required to execute the cross-language transform
func UnnamedInput ¶
func UnnamedInput(col PCollection) map[string]PCollection
UnnamedInput is a helper function for passing single unnamed inputs to `beam.CrossLanguage`.
Example:
beam.CrossLanguage(s, urn, payload, addr, UnnamedInput(input), outputs);
func UnnamedOutput ¶
UnnamedOutput is a helper function for passing single unnamed output types to `beam.CrossLanguage`.
Example:
beam.CrossLanguage(s, urn, payload, addr, inputs, UnnamedOutput(output));
func ValidateKVType ¶
func ValidateKVType(col PCollection) (typex.FullType, typex.FullType)
ValidateKVType panics if the type of the PCollection is not KV<A,B>. It returns (A,B).
func ValidateNonCompositeType ¶
func ValidateNonCompositeType(col PCollection) typex.FullType
ValidateNonCompositeType panics if the type of the PCollection is not a composite type. It returns the type.
Types ¶
type Coder ¶
type Coder struct {
// contains filtered or unexported fields
}
Coder defines how to encode and decode values of type 'A' into byte streams. Coders are attached to PCollections of the same type. For PCollections consumed by GBK, the attached coders are required to be deterministic.
func DecodeCoder ¶
DecodeCoder decodes a coder. Any custom coder function symbol must be resolvable via the runtime.GlobalSymbolResolver. The types must be encodable.
type Counter ¶
Counter is a metric that can be incremented and decremented, and is aggregated by the sum.
Counters are safe to use in multiple bundles simultaneously, but not generally threadsafe. Your DoFn needs to manage the thread safety of Beam metrics for any additional concurrency it uses.
func NewCounter ¶
NewCounter returns the Counter with the given namespace and name.
func (Counter) Dec ¶
Dec decrements the counter within by the given amount.
Example ¶
package main import ( "context" "github.com/apache/beam/sdks/go/pkg/beam" ) // A beam_test global context var to improve how the examples look. var ctx = context.Background() func main() { c := beam.NewCounter("example", "size") c.Dec(ctx, int64(len("foobar"))) }
Output:
func (Counter) Inc ¶
Inc increments the counter within by the given amount.
Example ¶
package main import ( "context" "github.com/apache/beam/sdks/go/pkg/beam" ) // A beam_test global context var to improve how the examples look. var ctx = context.Background() func main() { c := beam.NewCounter("example", "size") c.Inc(ctx, int64(len("foobar"))) }
Output:
type Distribution ¶
type Distribution struct {
*metrics.Distribution
}
Distribution is a metric that records various statistics about the distribution of reported values.
Distributions are safe to use in multiple bundles simultaneously, but not generally threadsafe. Your DoFn needs to manage the thread safety of Beam metrics for any additional concurrency it uses.
func NewDistribution ¶
func NewDistribution(namespace, name string) Distribution
NewDistribution returns the Distribution with the given namespace and name.
func (Distribution) Update ¶
func (c Distribution) Update(ctx context.Context, v int64)
Update adds an observation to this distribution.
Example ¶
package main import ( "context" "time" "github.com/apache/beam/sdks/go/pkg/beam" ) // A beam_test global context var to improve how the examples look. var ctx = context.Background() func main() { t := time.Millisecond * 42 d := beam.NewDistribution("example", "latency_micros") d.Update(ctx, int64(t/time.Microsecond)) }
Output:
type ElementDecoder ¶
type ElementDecoder = coder.ElementDecoder
ElementDecoder encapsulates being able to decode an element from a reader.
func NewElementDecoder ¶
func NewElementDecoder(t reflect.Type) ElementDecoder
NewElementDecoder returns an ElementDecoder the given type.
type ElementEncoder ¶
type ElementEncoder = coder.ElementEncoder
ElementEncoder encapsulates being able to encode an element into a writer.
func NewElementEncoder ¶
func NewElementEncoder(t reflect.Type) ElementEncoder
NewElementEncoder returns a new encoding function for the given type.
type EncodedCoder ¶
type EncodedCoder struct { // Coder is the coder to preserve across serialization. Coder Coder }
EncodedCoder is a serialization wrapper around a coder for convenience.
func (EncodedCoder) MarshalJSON ¶
func (w EncodedCoder) MarshalJSON() ([]byte, error)
MarshalJSON returns the JSON encoding this value.
func (*EncodedCoder) UnmarshalJSON ¶
func (w *EncodedCoder) UnmarshalJSON(buf []byte) error
UnmarshalJSON sets the state of this instance from the passed in JSON.
type EncodedFunc ¶
type EncodedFunc struct { // Fn is the function to preserve across serialization. Fn reflectx.Func }
EncodedFunc is a serialization wrapper around a function for convenience.
func (EncodedFunc) MarshalJSON ¶
func (w EncodedFunc) MarshalJSON() ([]byte, error)
MarshalJSON returns the JSON encoding this value.
func (*EncodedFunc) UnmarshalJSON ¶
func (w *EncodedFunc) UnmarshalJSON(buf []byte) error
UnmarshalJSON sets the state of this instance from the passed in JSON.
type EncodedType ¶
EncodedType is a serialization wrapper around a type for convenience.
func (EncodedType) MarshalJSON ¶
func (w EncodedType) MarshalJSON() ([]byte, error)
MarshalJSON returns the JSON encoding this value.
func (*EncodedType) UnmarshalJSON ¶
func (w *EncodedType) UnmarshalJSON(buf []byte) error
UnmarshalJSON sets the state of this instance from the passed in JSON.
type EventTime ¶
EventTime represents the time of the event that generated an element. This is distinct from the time when an element is processed.
type FullType ¶
FullType represents the tree structure of data types processed by the graph. It allows representation of composite types, such as KV<int, string> or CoGBK<int, int>, as well as "generic" such types, KV<int,T> or CoGBK<X,Y>, where the free "type variables" are the fixed universal types: T, X, etc.
type Gauge ¶
Gauge is a metric that can have its new value set, and is aggregated by taking the last reported value.
Gauge are safe to use in multiple bundles simultaneously, but not generally threadsafe. Your DoFn needs to manage the thread safety of Beam metrics for any additional concurrency it uses.
func (Gauge) Set ¶
Set sets the current value for this gauge.
Example ¶
package main import ( "context" "github.com/apache/beam/sdks/go/pkg/beam" ) // A beam_test global context var to improve how the examples look. var ctx = context.Background() func main() { g := beam.NewGauge("example", "progress") g.Set(ctx, 42) }
Output:
type Option ¶
type Option interface {
// contains filtered or unexported methods
}
Option is an optional value or context to a transformation, used at pipeline construction time. The primary use case is providing side inputs.
type PCollection ¶
type PCollection struct {
// contains filtered or unexported fields
}
PCollection is an immutable collection of values of type 'A', which must be a concrete type, such as int or KV<int,string>. A PCollection can contain either a bounded or unbounded number of elements. Bounded and unbounded PCollections are produced as the output of PTransforms (including root PTransforms like textio.Read), and can be passed as the inputs of other PTransforms. Some root transforms produce bounded PCollections and others produce unbounded ones.
Each element in a PCollection has an associated timestamp. Sources assign timestamps to elements when they create PCollections, and other PTransforms propagate these timestamps from their input to their output implicitly or explicitly.
Additionally, each element is assigned to a set of windows. By default, all elements are assigned into a single default window, GlobalWindow.
func AddFixedKey ¶
func AddFixedKey(s Scope, col PCollection) PCollection
AddFixedKey adds a fixed key (0) to every element.
func CoGroupByKey ¶
func CoGroupByKey(s Scope, cols ...PCollection) PCollection
CoGroupByKey inserts a CoGBK transform into the pipeline.
func Combine ¶
func Combine(s Scope, combinefn interface{}, col PCollection, opts ...Option) PCollection
Combine inserts a global Combine transform into the pipeline. It expects a PCollection<T> as input where T is a concrete type. Combine supports TypeDefinition options for binding generic types in combinefn.
func CombinePerKey ¶
func CombinePerKey(s Scope, combinefn interface{}, col PCollection, opts ...Option) PCollection
CombinePerKey inserts a GBK and per-key Combine transform into the pipeline. It expects a PCollection<KV<K,T>>. The CombineFn may optionally take a key parameter. CombinePerKey supports TypeDefinition options for binding generic types in combinefn.
func Create ¶
func Create(s Scope, values ...interface{}) PCollection
Create inserts a fixed non-empty set of values into the pipeline. The values must be of the same type 'A' and the returned PCollection is of type A.
The returned PCollections can be used as any other PCollections. The values are JSON-coded. Each runner may place limits on the sizes of the values and Create should generally only be used for small collections.
Example ¶
package main import ( "github.com/apache/beam/sdks/go/pkg/beam" ) var s = beam.Scope{} func main() { beam.Create(s, 5, 6, 7, 8, 9) // PCollection<int> beam.Create(s, []int{5, 6}, []int{7, 8, 9}) // PCollection<[]int> beam.Create(s, []int{5, 6, 7, 8, 9}) // PCollection<[]int> beam.Create(s, "a", "b", "c") // PCollection<string> }
Output:
func CreateList ¶
func CreateList(s Scope, list interface{}) PCollection
CreateList inserts a fixed set of values into the pipeline from a slice or array. Unlike Create this supports the creation of an empty PCollection.
Example ¶
package main import ( "github.com/apache/beam/sdks/go/pkg/beam" ) var s = beam.Scope{} func main() { beam.CreateList(s, []int{5, 6, 7, 8, 9}) // PCollection<int> }
Output:
func DropKey ¶
func DropKey(s Scope, col PCollection) PCollection
DropKey drops the key for an input PCollection<KV<A,B>>. It returns a PCollection<B>.
func DropValue ¶
func DropValue(s Scope, col PCollection) PCollection
DropValue drops the value for an input PCollection<KV<A,B>>. It returns a PCollection<A>.
func Explode ¶
func Explode(s Scope, col PCollection) PCollection
Explode is a PTransform that takes a single PCollection<[]A> and returns a PCollection<A> containing all the elements for each incoming slice.
Example ¶
package main import ( "github.com/apache/beam/sdks/go/pkg/beam" ) var s = beam.Scope{} func main() { d := beam.Create(s, []int{1, 2, 3, 4, 5}) // PCollection<[]int> beam.Explode(s, d) // PCollection<int> }
Output:
func External ¶
func External(s Scope, spec string, payload []byte, in []PCollection, out []FullType, bounded bool) []PCollection
External defines a Beam external transform. The interpretation of this primitive is runner specific. The runner is responsible for parsing the payload based on the spec provided to implement the behavior of the operation. Transform libraries should expose an API that captures the user's intent and serialize the payload as a byte slice that the runner will deserialize.
func Flatten ¶
func Flatten(s Scope, cols ...PCollection) PCollection
Flatten is a PTransform that takes either multiple PCollections of type 'A' and returns a single PCollection of type 'A' containing all the elements in all the input PCollections. The name "Flatten" suggests taking a list of lists and flattening them into a single list.
By default, the Coder of the output PCollection is the same as the Coder of the first PCollection.
Example ¶
package main import ( "github.com/apache/beam/sdks/go/pkg/beam" "github.com/apache/beam/sdks/go/pkg/beam/io/textio" ) var s = beam.Scope{} func main() { a := textio.Read(s, "...some file path...") // PCollection<string> b := textio.Read(s, "...some other file path...") c := textio.Read(s, "...some third file path...") beam.Flatten(s, a, b, c) // PCollection<String> }
Output:
func GroupByKey ¶
func GroupByKey(s Scope, a PCollection) PCollection
GroupByKey is a PTransform that takes a PCollection of type KV<A,B>, groups the values by key and windows, and returns a PCollection of type GBK<A,B> representing a map from each distinct key and window of the input PCollection to an iterable over all the values associated with that key in the input per window. Each key in the output PCollection is unique within each window.
GroupByKey is analogous to converting a multi-map into a uni-map, and related to GROUP BY in SQL. It corresponds to the "shuffle" step between the Mapper and the Reducer in the MapReduce framework.
Two keys of type A are compared for equality by first encoding each of the keys using the Coder of the keys of the input PCollection, and then comparing the encoded bytes. This admits efficient parallel evaluation. Note that this requires that the Coder of the keys be deterministic.
By default, input and output PCollections share a key Coder and iterable values in the input and output PCollection share an element Coder.
GroupByKey is a key primitive in data-parallel processing, since it is the main way to efficiently bring associated data together into one location. It is also a key determiner of the performance of a data-parallel pipeline.
See CoGroupByKey for a way to group multiple input PCollections by a common key at once.
Example ¶
package main import ( "github.com/apache/beam/sdks/go/pkg/beam" ) var s = beam.Scope{} func main() { type Doc struct{} var urlDocPairs beam.PCollection // PCollection<KV<string, Doc>> urlToDocs := beam.GroupByKey(s, urlDocPairs) // PCollection<CoGBK<string, Doc>> // CoGBK parameters receive an iterator function with all values associated // with the same key. beam.ParDo0(s, func(key string, values func(*Doc) bool) { var cur Doc for values(&cur) { // ... process all docs having that url ... } }, urlToDocs) // PCollection<KV<string, []Doc>> }
Output:
func Impulse ¶
func Impulse(s Scope) PCollection
Impulse emits a single empty []byte into the global window. The resulting PCollection is a singleton of type []byte.
The purpose of Impulse is to trigger another transform, such as ones that take all information as side inputs.
Example ¶
package main import ( "github.com/apache/beam/sdks/go/pkg/beam" ) var s = beam.Scope{} func main() { beam.Impulse(s) // PCollection<[]byte> }
Output:
func ImpulseValue ¶
func ImpulseValue(s Scope, value []byte) PCollection
ImpulseValue emits the supplied byte slice into the global window. The resulting PCollection is a singleton of type []byte.
Example ¶
package main import ( "github.com/apache/beam/sdks/go/pkg/beam" ) var s = beam.Scope{} func main() { beam.ImpulseValue(s, []byte{}) // PCollection<[]byte> }
Output:
func Must ¶
func Must(a PCollection, err error) PCollection
Must returns the input, but panics if err != nil.
func MustN ¶
func MustN(list []PCollection, err error) []PCollection
MustN returns the input, but panics if err != nil.
func ParDo ¶
func ParDo(s Scope, dofn interface{}, col PCollection, opts ...Option) PCollection
ParDo is the core element-wise PTransform in Apache Beam, invoking a user-specified function on each of the elements of the input PCollection to produce zero or more output elements, all of which are collected into the output PCollection. Use one of the ParDo variants for a different number of output PCollections. The PCollections do no need to have the same types.
Elements are processed independently, and possibly in parallel across distributed cloud resources. The ParDo processing style is similar to what happens inside the "Mapper" or "Reducer" class of a MapReduce-style algorithm.
DoFns ¶
The function to use to process each element is specified by a DoFn, either as single function or as a struct with methods, notably ProcessElement. The struct may also define Setup, StartBundle, FinishBundle and Teardown methods. The struct is JSON-serialized and may contain construction-time values.
Conceptually, when a ParDo transform is executed, the elements of the input PCollection are first divided up into some number of "bundles". These are farmed off to distributed worker machines (or run locally, if using the direct runner). For each bundle of input elements processing proceeds as follows:
- If a struct, a fresh instance of the argument DoFn is created on a worker from json serialization, and the Setup method is called on this instance, if present. A runner may reuse DoFn instances for multiple bundles. A DoFn that has terminated abnormally (by returning an error) will never be reused.
- The DoFn's StartBundle method, if provided, is called to initialize it.
- The DoFn's ProcessElement method is called on each of the input elements in the bundle.
- The DoFn's FinishBundle method, if provided, is called to complete its work. After FinishBundle is called, the framework will not again invoke ProcessElement or FinishBundle until a new call to StartBundle has occurred.
- If any of Setup, StartBundle, ProcessElement or FinishBundle methods return an error, the Teardown method, if provided, will be called on the DoFn instance.
- If a runner will no longer use a DoFn, the Teardown method, if provided, will be called on the discarded instance.
Each of the calls to any of the DoFn's processing methods can produce zero or more output elements. All of the of output elements from all of the DoFn instances are included in an output PCollection.
For example:
words := beam.ParDo(s, &Foo{...}, ...) lengths := beam.ParDo(s, func (word string) int) { return len(word) }, words)
Each output element has the same timestamp and is in the same windows as its corresponding input element. The timestamp can be accessed and/or emitted by including a EventTime-typed parameter. The name of the function or struct is used as the DoFn name. Function literals do not have stable names and should thus not be used in production code.
Side Inputs ¶
While a ParDo processes elements from a single "main input" PCollection, it can take additional "side input" PCollections. These SideInput along with the DoFn parameter form express styles of accessing PCollection computed by earlier pipeline operations, passed in to the ParDo transform using SideInput options, and their contents accessible to each of the DoFn operations. For example:
words := ... cufoff := ... // Singleton PCollection<int> smallWords := beam.ParDo(s, func (word string, cutoff int, emit func(string)) { if len(word) < cutoff { emit(word) } }, words, beam.SideInput{Input: cutoff})
Additional Outputs ¶
Optionally, a ParDo transform can produce zero or multiple output PCollections. Note the use of ParDo2 to specfic 2 outputs. For example:
words := ... cufoff := ... // Singleton PCollection<int> small, big := beam.ParDo2(s, func (word string, cutoff int, small, big func(string)) { if len(word) < cutoff { small(word) } else { big(word) } }, words, beam.SideInput{Input: cutoff})
By default, the Coders for the elements of each output PCollections is inferred from the concrete type.
No Global Shared State ¶
There are three main ways to initialize the state of a DoFn instance processing a bundle:
Define public instance variable state. This state will be automatically JSON serialized and then deserialized in the DoFn instances created for bundles. This method is good for state known when the original DoFn is created in the main program, if it's not overly large. This is not suitable for any state which must only be used for a single bundle, as DoFn's may be used to process multiple bundles.
Compute the state as a singleton PCollection and pass it in as a side input to the DoFn. This is good if the state needs to be computed by the pipeline, or if the state is very large and so is best read from file(s) rather than sent as part of the DoFn's serialized state.
Initialize the state in each DoFn instance, in a StartBundle method. This is good if the initialization doesn't depend on any information known only by the main program or computed by earlier pipeline operations, but is the same for all instances of this DoFn for all program executions, say setting up empty caches or initializing constant data.
ParDo operations are intended to be able to run in parallel across multiple worker machines. This precludes easy sharing and updating mutable state across those machines. There is no support in the Beam model for communicating and synchronizing updates to shared state across worker machines, so programs should not access any mutable global variable state in their DoFn, without understanding that the Go processes for the main program and workers will each have its own independent copy of such state, and there won't be any automatic copying of that state across Java processes. All information should be communicated to DoFn instances via main and side inputs and serialized state, and all output should be communicated from a DoFn instance via output PCollections, in the absence of external communication mechanisms written by user code.
Splittable DoFns (Experimental)
Warning: Splittable DoFns are still experimental, largely untested, and likely to have bugs.
Splittable DoFns are DoFns that are able to split work within an element, as opposed to only at element boundaries like normal DoFns. This is useful for DoFns that emit many outputs per input element and can distribute that work among multiple workers. The most common examples of this are sources.
In order to split work within an element, splittable DoFns use the concept of restrictions, which are objects that are associated with an element and describe a portion of work on that element. For example, a restriction associated with a filename might describe what byte range within that file to process. In addition to restrictions, splittable DoFns also rely on restriction trackers to track progress and perform splits on a restriction currently being processed. See the `RTracker` interface in core/sdf/sdf.go for more details.
Splitting ¶
Splitting means taking one restriction and splitting into two or more that cover the entire input space of the original one. In other words, processing all the split restrictions should produce identical output to processing the original one.
Splitting occurs in two stages. The initial splitting occurs before any restrictions have started processing. This step is used to split large restrictions into smaller ones that can then be distributed among multiple workers for processing. Initial splitting is user-defined and optional.
Dynamic splitting occurs during the processing of a restriction in runners that have implemented it. If there are available workers, runners may split the unprocessed portion of work from a busy worker and shard it to available workers in order to better distribute work. With unsplittable DoFns this can only occur on element boundaries, but for splittable DoFns this split can land within a restriction and will require splitting that restriction.
- Note: The Go SDK currently does not support dynamic splitting for SDFs, only initial splitting. Only initially split restrictions can be distributed by liquid sharding. Stragglers will not be split during execution with dynamic splitting.
Splittable DoFn Methods ¶
Making a splittable DoFn requires the following methods to be implemented on a DoFn in addition to the usual DoFn requirements. In the following method signatures `elem` represents the main input elements to the DoFn, and should match the types used in ProcessElement. `restriction` represents the user-defined restriction, and can be any type as long as it is consistent throughout all the splittable DoFn methods:
- `CreateInitialRestriction(element) restriction` CreateInitialRestriction creates an initial restriction encompassing an entire element. The restriction created stays associated with the element it describes.
- `SplitRestriction(elem, restriction) []restriction` SplitRestriction takes an element and its initial restriction, and optionally performs an initial split on it, returning a slice of all the split restrictions. If no splits are desired, the method returns a slice containing only the original restriction. This method will always be called on each newly created restriction before they are processed.
- `RestrictionSize(elem, restriction) float64` RestrictionSize returns a cheap size estimation for a restriction. This size is an abstract scalar value that represents how much work a restriction takes compared to other restrictions in the same DoFn. For example, a size of 200 represents twice as much work as a size of 100, but the numbers do not represent anything on their own. Size is used by runners to estimate work for liquid sharding.
- `CreateTracker(restriction) restrictionTracker` CreateTracker creates and returns a restriction tracker (a concrete type implementing the `sdf.RTracker` interface) given a restriction. The restriction tracker is used to track progress processing a restriction, and to allow for dynamic splits. This method is called on each restriction right before processing begins.
- `ProcessElement(sdf.RTracker, element, func emit(output))` For splittable DoFns, ProcessElement requires a restriction tracker before inputs, and generally requires emits to be used for outputs, since restrictions will generally produce multiple outputs. For more details on processing restrictions in a splittable DoFn, see `sdf.RTracker`.
Fault Tolerance ¶
In a distributed system, things can fail: machines can crash, machines can be unable to communicate across the network, etc. While individual failures are rare, the larger the job, the greater the chance that something, somewhere, will fail. Beam runners may strive to mask such failures by retrying failed DoFn bundles. This means that a DoFn instance might process a bundle partially, then crash for some reason, then be rerun (often as a new process) on that same bundle and on the same elements as before. Sometimes two or more DoFn instances will be running on the same bundle simultaneously, with the system taking the results of the first instance to complete successfully. Consequently, the code in a DoFn needs to be written such that these duplicate (sequential or concurrent) executions do not cause problems. If the outputs of a DoFn are a pure function of its inputs, then this requirement is satisfied. However, if a DoFn's execution has external side-effects, such as performing updates to external HTTP services, then the DoFn's code needs to take care to ensure that those updates are idempotent and that concurrent updates are acceptable. This property can be difficult to achieve, so it is advisable to strive to keep DoFns as pure functions as much as possible.
Optimization ¶
Beam runners may choose to apply optimizations to a pipeline before it is executed. A key optimization, fusion, relates to ParDo operations. If one ParDo operation produces a PCollection that is then consumed as the main input of another ParDo operation, the two ParDo operations will be fused together into a single ParDo operation and run in a single pass; this is "producer-consumer fusion". Similarly, if two or more ParDo operations have the same PCollection main input, they will be fused into a single ParDo that makes just one pass over the input PCollection; this is "sibling fusion".
If after fusion there are no more unfused references to a PCollection (e.g., one between a producer ParDo and a consumer ParDo), the PCollection itself is "fused away" and won't ever be written to disk, saving all the I/O and space expense of constructing it.
When Beam runners apply fusion optimization, it is essentially "free" to write ParDo operations in a very modular, composable style, each ParDo operation doing one clear task, and stringing together sequences of ParDo operations to get the desired overall effect. Such programs can be easier to understand, easier to unit-test, easier to extend and evolve, and easier to reuse in new programs. The predefined library of PTransforms that come with Beam makes heavy use of this modular, composable style, trusting to the runner to "flatten out" all the compositions into highly optimized stages.
See https://beam.apache.org/documentation/programming-guide/#pardo for the web documentation for ParDo
Example (AdditionalOutputs) ¶
Optionally, a ParDo transform can produce zero or multiple output PCollections. Note the use of ParDo2 to specify 2 outputs.
package main import ( "github.com/apache/beam/sdks/go/pkg/beam" ) var s = beam.Scope{} func main() { var words beam.PCollection // PCollection<string> var cutoff beam.PCollection // Singleton PCollection<int> small, big := beam.ParDo2(s, func(word string, cutoff int, small, big func(string)) { if len(word) < cutoff { small(word) } else { big(word) } }, words, beam.SideInput{Input: cutoff}) _, _ = small, big }
Output:
func ParDoN ¶
func ParDoN(s Scope, dofn interface{}, col PCollection, opts ...Option) []PCollection
ParDoN inserts a ParDo with any number of outputs into the pipeline.
func Partition ¶
func Partition(s Scope, n int, fn interface{}, col PCollection) []PCollection
Partition takes a PCollection<T> and a PartitionFn, uses the PartitionFn to split the elements of the input PCollection into N partitions, and returns a []PCollection<T> that bundles N PCollection<T>s containing the split elements.
A PartitionFn has the signature `func(T) int.`
T is permitted to be a KV.
func Reshuffle ¶
func Reshuffle(s Scope, col PCollection) PCollection
Reshuffle copies a PCollection of the same kind and using the same element coder, and maintains the same windowing information. Importantly, it allows the result PCollection to be processed with a different sharding, in a different stage than the input PCollection.
For example, if a computation needs a lot of parallelism but produces only a small amount of output data, then the computation producing the data can run with as much parallelism as needed, while the output file is written with a smaller amount of parallelism, using the following pattern:
pc := bigHairyComputationNeedingParallelism(scope) // PCollection<string> resharded := beam.Reshuffle(scope, pc) // PCollection<string>
Another use case is when one has a non-deterministic DoFn followed by one that performs externally-visible side effects. Inserting a Reshuffle between these DoFns ensures that retries of the second DoFn will always be the same, which is necessary to make side effects idempotent.
A Reshuffle will force a break in the optimized pipeline. Consequently, this operation should be used sparingly, only after determining that the pipeline without reshuffling is broken in some way and performing an extra operation is worth the cost.
func Seq ¶
func Seq(s Scope, col PCollection, dofns ...interface{}) PCollection
Seq is a convenience helper to chain single-input/single-output ParDos together in a sequence.
Example ¶
package main import ( "math" "strconv" "github.com/apache/beam/sdks/go/pkg/beam" "github.com/apache/beam/sdks/go/pkg/beam/io/textio" ) var s = beam.Scope{} func main() { a := textio.Read(s, "...some file path...") // PCollection<string> beam.Seq(s, a, strconv.Atoi, // string to int func(i int) float64 { return float64(i) }, // int to float64 math.Signbit, // float64 to bool ) // PCollection<bool> }
Output:
func SwapKV ¶
func SwapKV(s Scope, col PCollection) PCollection
SwapKV swaps the key and value for an input PCollection<KV<A,B>>. It returns a PCollection<KV<B,A>>.
func TryCoGroupByKey ¶
func TryCoGroupByKey(s Scope, cols ...PCollection) (PCollection, error)
TryCoGroupByKey inserts a CoGBK transform into the pipeline. Returns an error on failure.
func TryCombine ¶
func TryCombine(s Scope, combinefn interface{}, col PCollection, opts ...Option) (PCollection, error)
TryCombine attempts to insert a global Combine transform into the pipeline. It may fail for multiple reasons, notably that the combinefn is not valid or cannot be bound -- due to type mismatch, say -- to the incoming PCollections.
func TryCombinePerKey ¶
func TryCombinePerKey(s Scope, combinefn interface{}, col PCollection, opts ...Option) (PCollection, error)
TryCombinePerKey attempts to insert a per-key Combine transform into the pipeline. It may fail for multiple reasons, notably that the combinefn is not valid or cannot be bound -- due to type mismatch, say -- to the incoming PCollection.
func TryCreate ¶
func TryCreate(s Scope, values ...interface{}) (PCollection, error)
TryCreate inserts a fixed non-empty set of values into the pipeline. The values must be of the same type.
func TryCreateList ¶
func TryCreateList(s Scope, list interface{}) (PCollection, error)
TryCreateList inserts a fixed set of values into the pipeline from a slice or array. The values must be of the same type. Unlike TryCreate this supports the creation of an empty PCollection.
func TryExternal ¶
func TryExternal(s Scope, spec string, payload []byte, in []PCollection, out []FullType, bounded bool) ([]PCollection, error)
TryExternal attempts to perform the work of External, returning an error indicating why the operation failed.
func TryFlatten ¶
func TryFlatten(s Scope, cols ...PCollection) (PCollection, error)
TryFlatten merges incoming PCollections of type 'A' to a single PCollection of type 'A'. Returns an error indicating the set of PCollections that could not be flattened.
func TryGroupByKey ¶
func TryGroupByKey(s Scope, a PCollection) (PCollection, error)
TryGroupByKey inserts a GBK transform into the pipeline. Returns an error on failure.
func TryParDo ¶
func TryParDo(s Scope, dofn interface{}, col PCollection, opts ...Option) ([]PCollection, error)
TryParDo attempts to insert a ParDo transform into the pipeline. It may fail for multiple reasons, notably that the dofn is not valid or cannot be bound -- due to type mismatch, say -- to the incoming PCollections.
func TryReshuffle ¶
func TryReshuffle(s Scope, col PCollection) (PCollection, error)
TryReshuffle inserts a Reshuffle into the pipeline, and returns an error if the pcollection's unable to be reshuffled.
func TryWindowInto ¶
func TryWindowInto(s Scope, ws *window.Fn, col PCollection) (PCollection, error)
TryWindowInto attempts to insert a WindowInto transform.
func WindowInto ¶
func WindowInto(s Scope, ws *window.Fn, col PCollection) PCollection
WindowInto applies the windowing strategy to each element.
func (PCollection) Coder ¶
func (p PCollection) Coder() Coder
Coder returns the coder for the collection. The Coder is of type 'A'.
func (PCollection) IsValid ¶
func (p PCollection) IsValid() bool
IsValid returns true iff the PCollection is valid and part of a Pipeline. Any use of an invalid PCollection will result in a panic.
func (PCollection) SetCoder ¶
func (p PCollection) SetCoder(c Coder) error
SetCoder set the coder for the collection. The Coder must be of type 'A'.
func (PCollection) String ¶
func (p PCollection) String() string
func (PCollection) Type ¶
func (p PCollection) Type() FullType
Type returns the full type 'A' of the elements. 'A' must be a concrete type, such as int or KV<int,string>.
type Pipeline ¶
type Pipeline struct {
// contains filtered or unexported fields
}
Pipeline manages a directed acyclic graph of primitive PTransforms, and the PCollections that the PTransforms consume and produce. Each Pipeline is self-contained and isolated from any other Pipeline. The Pipeline owns the PCollections and PTransforms and they can be used by that Pipeline only. Pipelines can safely be executed concurrently.
func (*Pipeline) Build ¶
Build validates the Pipeline and returns a lower-level representation for execution. It is called by runners only.
type PipelineResult ¶
PipelineResult is the result of beamx.RunWithMetrics.
type SchemaProvider ¶
type SchemaProvider interface { FromLogicalType(reflect.Type) (reflect.Type, error) BuildEncoder(rt reflect.Type) (func(interface{}, io.Writer) error, error) BuildDecoder(rt reflect.Type) (func(io.Reader) (interface{}, error), error) }
SchemaProvider specializes schema handling for complex types, including conversion to a valid schema base type,
In particular, they are intended to handle schema for interface types.
Sepearated out the acting type from the provider implementation is good.
type Scope ¶
type Scope struct {
// contains filtered or unexported fields
}
Scope is a hierarchical grouping for composite transforms. Scopes can be enclosed in other scopes and for a tree structure. For pipeline updates, the scope chain form a unique name. The scope chain can also be used for monitoring and visualization purposes.
func (Scope) IsValid ¶
IsValid returns true iff the Scope is valid. Any use of an invalid Scope will result in a panic.
type SideInput ¶
type SideInput struct {
Input PCollection
}
SideInput provides a view of the given PCollection to the transformation.
Example ¶
package main import ( "github.com/apache/beam/sdks/go/pkg/beam" ) var s = beam.Scope{} func main() { // words and sample are PCollection<string> var words, sample beam.PCollection // analyzeFn emits values from the primary based on the singleton side input. analyzeFn := func(primary string, side string, emit func(string)) {} // Use beam.SideInput to declare that the sample PCollection is the side input. beam.ParDo(s, analyzeFn, words, beam.SideInput{Input: sample}) }
Output:
type T ¶
T is a Universal Type used to represent "generic" types in DoFn and PCollection signatures. Each universal type is distinct from all others.
type TypeDefinition ¶
type TypeDefinition struct { // Var is the universal type defined. Var reflect.Type // T is the type it is bound to. T reflect.Type }
TypeDefinition provides construction-time type information that the platform cannot infer, such as structured storage sources. These types are universal types that appear as output only. Types that are inferrable should not be conveyed via this mechanism.
type U ¶
U is a Universal Type used to represent "generic" types in DoFn and PCollection signatures. Each universal type is distinct from all others.
type V ¶
V is a Universal Type used to represent "generic" types in DoFn and PCollection signatures. Each universal type is distinct from all others.
type W ¶
W is a Universal Type used to represent "generic" types in DoFn and PCollection signatures. Each universal type is distinct from all others.
type Window ¶
Window represents the aggregation window of this element. An element can be a part of multiple windows, based on the element's event time.
type X ¶
X is a Universal Type used to represent "generic" types in DoFn and PCollection signatures. Each universal type is distinct from all others.
Source Files ¶
Directories ¶
Path | Synopsis |
---|---|
Package artifact contains utilities for staging and retrieving artifacts.
|
Package artifact contains utilities for staging and retrieving artifacts. |
gcsproxy
Package gcsproxy contains artifact staging and retrieval servers backed by GCS.
|
Package gcsproxy contains artifact staging and retrieval servers backed by GCS. |
Package core contains constants and other static data related to the SDK, such as the SDK Name and version.
|
Package core contains constants and other static data related to the SDK, such as the SDK Name and version. |
funcx
Package funcx contains functions and types used to perform type analysis of Beam functions.
|
Package funcx contains functions and types used to perform type analysis of Beam functions. |
graph
Package graph is the internal representation of the Beam execution plan.
|
Package graph is the internal representation of the Beam execution plan. |
graph/coder
Package coder contains coder representation and utilities.
|
Package coder contains coder representation and utilities. |
graph/coder/testutil
Package testutil contains helpers to test and validate custom Beam Schema coders.
|
Package testutil contains helpers to test and validate custom Beam Schema coders. |
graph/mtime
Package mtime contains a millisecond representation of time.
|
Package mtime contains a millisecond representation of time. |
graph/window
Package window contains window representation, windowing strategies and utilities.
|
Package window contains window representation, windowing strategies and utilities. |
metrics
Package metrics implements the Beam metrics API, described at http://s.apache.org/beam-metrics-api Metrics in the Beam model are uniquely identified by a namespace, a name, and the PTransform context in which they are used.
|
Package metrics implements the Beam metrics API, described at http://s.apache.org/beam-metrics-api Metrics in the Beam model are uniquely identified by a namespace, a name, and the PTransform context in which they are used. |
runtime
Package runtime contains runtime hooks and utilities for pipeline options and type registration.
|
Package runtime contains runtime hooks and utilities for pipeline options and type registration. |
runtime/coderx
Package coderx contains coders for primitive types that aren't included in the beam model.
|
Package coderx contains coders for primitive types that aren't included in the beam model. |
runtime/exec
Package exec contains runtime plan representation and execution.
|
Package exec contains runtime plan representation and execution. |
runtime/exec/optimized
Package optimized contains type-specialized shims for faster execution.
|
Package optimized contains type-specialized shims for faster execution. |
runtime/genx
Package genx is a convenience package to better support the code generator.
|
Package genx is a convenience package to better support the code generator. |
runtime/graphx
Package graphx provides facilities to help with the serialization of pipelines into a serializable graph structure suitable for the worker.
|
Package graphx provides facilities to help with the serialization of pipelines into a serializable graph structure suitable for the worker. |
runtime/graphx/schema
Package schema contains utility functions for relating Go types and Beam Schemas.
|
Package schema contains utility functions for relating Go types and Beam Schemas. |
runtime/graphx/v1
Package v1 is a generated protocol buffer package.
|
Package v1 is a generated protocol buffer package. |
runtime/harness
Package harness implements the SDK side of the Beam FnAPI.
|
Package harness implements the SDK side of the Beam FnAPI. |
runtime/harness/init
Package init contains the harness initialization code defined by the FnAPI.
|
Package init contains the harness initialization code defined by the FnAPI. |
runtime/harness/session
Package session is a generated protocol buffer package.
|
Package session is a generated protocol buffer package. |
runtime/pipelinex
Package pipelinex contains utilities for manipulating Beam proto pipelines.
|
Package pipelinex contains utilities for manipulating Beam proto pipelines. |
sdf
Package contains interfaces used specifically for splittable DoFns.
|
Package contains interfaces used specifically for splittable DoFns. |
typex
Package typex contains full type representation for PCollections and DoFns, and utilities for type checking.
|
Package typex contains full type representation for PCollections and DoFns, and utilities for type checking. |
util/dot
Package dot produces DOT graphs from Beam graph representations.
|
Package dot produces DOT graphs from Beam graph representations. |
util/hooks
Package hooks allows runners to tailor execution of the worker harness.
|
Package hooks allows runners to tailor execution of the worker harness. |
util/ioutilx
Package ioutilx contains additional io utilities.
|
Package ioutilx contains additional io utilities. |
util/jsonx
Package jsonx contains utilities for working with JSON encoded data.
|
Package jsonx contains utilities for working with JSON encoded data. |
util/protox
Package protox contains utilities for working with protobufs.
|
Package protox contains utilities for working with protobufs. |
util/reflectx
Package reflectx contains a set of reflection utilities and well-known types.
|
Package reflectx contains a set of reflection utilities and well-known types. |
util/stringx
Package stringx contains utilities for working with strings.
|
Package stringx contains utilities for working with strings. |
util/symtab
Package symtab allows reading low-level symbol information from the symbol table.
|
Package symtab allows reading low-level symbol information from the symbol table. |
internal
|
|
errors
Package errors contains functionality for creating and wrapping errors with improved formatting compared to the standard Go error functionality.
|
Package errors contains functionality for creating and wrapping errors with improved formatting compared to the standard Go error functionality. |
io
|
|
avroio
Package avroio contains transforms for reading and writing avro files.
|
Package avroio contains transforms for reading and writing avro files. |
bigqueryio
Package bigqueryio provides transformations and utilities to interact with Google BigQuery.
|
Package bigqueryio provides transformations and utilities to interact with Google BigQuery. |
databaseio
Package databaseio provides transformations and utilities to interact with generic database database/sql API.
|
Package databaseio provides transformations and utilities to interact with generic database database/sql API. |
datastoreio
Package datastoreio provides transformations and utilities to interact with Google Datastore.
|
Package datastoreio provides transformations and utilities to interact with Google Datastore. |
filesystem
Package filesystem contains an extensible file system abstraction.
|
Package filesystem contains an extensible file system abstraction. |
filesystem/gcs
Package gcs contains a Google Cloud Storage (GCS) implementation of the Beam file system.
|
Package gcs contains a Google Cloud Storage (GCS) implementation of the Beam file system. |
filesystem/local
Package local contains a local file implementation of the Beam file system.
|
Package local contains a local file implementation of the Beam file system. |
filesystem/memfs
Package memfs contains a in-memory Beam filesystem.
|
Package memfs contains a in-memory Beam filesystem. |
pubsubio
Package pubsubio provides access to PubSub on Dataflow streaming.
|
Package pubsubio provides access to PubSub on Dataflow streaming. |
pubsubio/v1
Package v1 is a generated protocol buffer package.
|
Package v1 is a generated protocol buffer package. |
rtrackers/offsetrange
Package offsetrange defines a restriction and restriction tracker for offset ranges.
|
Package offsetrange defines a restriction and restriction tracker for offset ranges. |
synthetic
Package synthetic contains transforms for creating synthetic pipelines.
|
Package synthetic contains transforms for creating synthetic pipelines. |
textio
Package textio contains transforms for reading and writing text files.
|
Package textio contains transforms for reading and writing text files. |
Package log contains a re-targetable context-aware logging system.
|
Package log contains a re-targetable context-aware logging system. |
Package model contains the portable Beam model contracts.
|
Package model contains the portable Beam model contracts. |
options
|
|
gcpopts
Package gcpopts contains shared options for Google Cloud Platform.
|
Package gcpopts contains shared options for Google Cloud Platform. |
jobopts
Package jobopts contains shared options for job submission.
|
Package jobopts contains shared options for job submission. |
Package provision contains utilities for obtaining runtime provision, information -- such as pipeline options.
|
Package provision contains utilities for obtaining runtime provision, information -- such as pipeline options. |
runners
|
|
dataflow
Package dataflow contains the Dataflow runner for submitting pipelines to Google Cloud Dataflow.
|
Package dataflow contains the Dataflow runner for submitting pipelines to Google Cloud Dataflow. |
dataflow/dataflowlib
Package dataflowlib translates a Beam pipeline model to the Dataflow API job model, for submission to Google Cloud Dataflow.
|
Package dataflowlib translates a Beam pipeline model to the Dataflow API job model, for submission to Google Cloud Dataflow. |
direct
Package direct contains the direct runner for running single-bundle pipelines in the current process.
|
Package direct contains the direct runner for running single-bundle pipelines in the current process. |
dot
Package dot is a Beam runner that "runs" a pipeline by producing a DOT graph of the execution plan.
|
Package dot is a Beam runner that "runs" a pipeline by producing a DOT graph of the execution plan. |
flink
Package flink contains the Flink runner.
|
Package flink contains the Flink runner. |
spark
Package spark contains the Spark runner.
|
Package spark contains the Spark runner. |
universal
Package universal contains a general-purpose runner that can submit jobs to any portable Beam runner.
|
Package universal contains a general-purpose runner that can submit jobs to any portable Beam runner. |
universal/extworker
Package extworker provides an external worker service and related utilities.
|
Package extworker provides an external worker service and related utilities. |
universal/runnerlib
Package runnerlib contains utilities for submitting Go pipelines to a Beam model runner.
|
Package runnerlib contains utilities for submitting Go pipelines to a Beam model runner. |
vet
Package vet is a Beam runner that "runs" a pipeline by producing generated code to avoid symbol table lookups and reflection in pipeline execution.
|
Package vet is a Beam runner that "runs" a pipeline by producing generated code to avoid symbol table lookups and reflection in pipeline execution. |
vet/testpipeline
Package testpipeline exports small test pipelines for testing the vet runner.
|
Package testpipeline exports small test pipelines for testing the vet runner. |
testing
|
|
passert
Package passert contains verification transformations for testing pipelines.
|
Package passert contains verification transformations for testing pipelines. |
ptest
Package ptest contains utilities for pipeline unit testing.
|
Package ptest contains utilities for pipeline unit testing. |
transforms
|
|
filter
Package filter contains transformations for removing pipeline elements based on various conditions.
|
Package filter contains transformations for removing pipeline elements based on various conditions. |
stats
Package stats contains transforms for statistical processing.
|
Package stats contains transforms for statistical processing. |
top
Package top contains transformations for finding the smallest (or largest) N elements based on arbitrary orderings.
|
Package top contains transformations for finding the smallest (or largest) N elements based on arbitrary orderings. |
util
|
|
errorx
Package errorx contains utilities for handling errors.
|
Package errorx contains utilities for handling errors. |
execx
Package execx contains wrappers and utilities for the exec package.
|
Package execx contains wrappers and utilities for the exec package. |
gcsx
Package gcsx contains utilities for working with Google Cloud Storage (GCS).
|
Package gcsx contains utilities for working with Google Cloud Storage (GCS). |
grpcx
Package grpcx contains utilities for working with gRPC.
|
Package grpcx contains utilities for working with gRPC. |
pubsubx
Package pubsubx contains utilities for working with Google PubSub.
|
Package pubsubx contains utilities for working with Google PubSub. |
shimx
Package shimx specifies the templates for generating type assertion shims for Apache Beam Go SDK pipelines.
|
Package shimx specifies the templates for generating type assertion shims for Apache Beam Go SDK pipelines. |
starcgenx
Package starcgenx is a Static Analysis Type Assertion shim and Registration Code Generator which provides an extractor to extract types from a package, in order to generate approprate shimsr a package so code can be generated for it.
|
Package starcgenx is a Static Analysis Type Assertion shim and Registration Code Generator which provides an extractor to extract types from a package, in order to generate approprate shimsr a package so code can be generated for it. |
syscallx
Package syscallx provides system call utilities that attempt to hide platform differences.
|
Package syscallx provides system call utilities that attempt to hide platform differences. |
x
|
|
beamx
Package beamx is a convenience package for beam.
|
Package beamx is a convenience package for beam. |
debug
Package debug contains pipeline components that may help in debugging pipeline issues.
|
Package debug contains pipeline components that may help in debugging pipeline issues. |
hooks/perf
Package perf is to add performance measuring hooks to a runner, such as cpu, heap, or trace profiles.
|
Package perf is to add performance measuring hooks to a runner, such as cpu, heap, or trace profiles. |