Skip to content

Commit

Permalink
Update parquetio and textio to work with -beam_strict (#26469)
Browse files Browse the repository at this point in the history
  • Loading branch information
jeremyje authored Apr 28, 2023
1 parent 0e176e3 commit 94d30aa
Show file tree
Hide file tree
Showing 2 changed files with 11 additions and 1 deletion.
10 changes: 9 additions & 1 deletion sdks/go/pkg/beam/io/parquetio/parquetio.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,15 +24,23 @@ import (

"github.com/apache/beam/sdks/v2/go/pkg/beam"
"github.com/apache/beam/sdks/v2/go/pkg/beam/io/filesystem"
"github.com/apache/beam/sdks/v2/go/pkg/beam/register"
"github.com/xitongsys/parquet-go-source/buffer"
"github.com/xitongsys/parquet-go/reader"
"github.com/xitongsys/parquet-go/writer"
)

func init() {
beam.RegisterFunction(expandFn)
register.Function3x1(expandFn)
register.Emitter1[string]()

beam.RegisterType(reflect.TypeOf((*parquetReadFn)(nil)).Elem())
register.DoFn3x1[context.Context, string, func(beam.X), error](&parquetReadFn{})
register.Emitter1[beam.X]()

beam.RegisterType(reflect.TypeOf((*parquetWriteFn)(nil)).Elem())
register.DoFn3x1[context.Context, int, func(*beam.X) bool, error](&parquetWriteFn{})
register.Iter1[beam.X]()
}

// Read reads a set of files and returns lines as a PCollection<elem>
Expand Down
2 changes: 2 additions & 0 deletions sdks/go/pkg/beam/io/textio/textio.go
Original file line number Diff line number Diff line change
Expand Up @@ -42,6 +42,8 @@ func init() {
register.Emitter2[string, string]()

beam.RegisterType(reflect.TypeOf((*writeFileFn)(nil)).Elem())
register.DoFn3x1[context.Context, int, func(*string) bool, error](&writeFileFn{})
register.Iter1[string]()
}

type readOption struct {
Expand Down

0 comments on commit 94d30aa

Please sign in to comment.