You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@beam.apache.org by "johannaojeling (via GitHub)" <gi...@apache.org> on 2023/03/12 09:54:44 UTC

[GitHub] [beam] johannaojeling opened a new pull request, #25809: [Go SDK]: Add fileio MatchFiles, MatchAll and ReadMatches transforms

johannaojeling opened a new pull request, #25809:
URL: https://github.com/apache/beam/pull/25809

   Fixes #25779 and adds the fileio transforms `MatchFiles`, `MatchAll` and `ReadMatches`
   
   ------------------------
   
   Thank you for your contribution! Follow this checklist to help us incorporate your contribution quickly and easily:
   
    - [ ] Mention the appropriate issue in your description (for example: `addresses #123`), if applicable. This will automatically add a link to the pull request in the issue. If you would like the issue to automatically close on merging the pull request, comment `fixes #<ISSUE NUMBER>` instead.
    - [ ] Update `CHANGES.md` with noteworthy changes.
    - [ ] If this contribution is large, please file an Apache [Individual Contributor License Agreement](https://www.apache.org/licenses/icla.pdf).
   
   See the [Contributor Guide](https://beam.apache.org/contribute) for more tips on [how to make review process smoother](https://beam.apache.org/contribute/get-started-contributing/#make-the-reviewers-job-easier).
   
   To check the build health, please visit [https://github.com/apache/beam/blob/master/.test-infra/BUILD_STATUS.md](https://github.com/apache/beam/blob/master/.test-infra/BUILD_STATUS.md)
   
   GitHub Actions Tests Status (on master branch)
   ------------------------------------------------------------------------------------------------
   [![Build python source distribution and wheels](https://github.com/apache/beam/workflows/Build%20python%20source%20distribution%20and%20wheels/badge.svg?branch=master&event=schedule)](https://github.com/apache/beam/actions?query=workflow%3A%22Build+python+source+distribution+and+wheels%22+branch%3Amaster+event%3Aschedule)
   [![Python tests](https://github.com/apache/beam/workflows/Python%20tests/badge.svg?branch=master&event=schedule)](https://github.com/apache/beam/actions?query=workflow%3A%22Python+Tests%22+branch%3Amaster+event%3Aschedule)
   [![Java tests](https://github.com/apache/beam/workflows/Java%20Tests/badge.svg?branch=master&event=schedule)](https://github.com/apache/beam/actions?query=workflow%3A%22Java+Tests%22+branch%3Amaster+event%3Aschedule)
   [![Go tests](https://github.com/apache/beam/workflows/Go%20tests/badge.svg?branch=master&event=schedule)](https://github.com/apache/beam/actions?query=workflow%3A%22Go+tests%22+branch%3Amaster+event%3Aschedule)
   
   See [CI.md](https://github.com/apache/beam/blob/master/CI.md) for more information about GitHub Actions CI.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] johannaojeling commented on a diff in pull request #25809: [Go SDK]: Add fileio MatchFiles, MatchAll and ReadMatches transforms

Posted by "johannaojeling (via GitHub)" <gi...@apache.org>.
johannaojeling commented on code in PR #25809:
URL: https://github.com/apache/beam/pull/25809#discussion_r1147858543


##########
sdks/go/pkg/beam/io/fileio/match.go:
##########
@@ -0,0 +1,181 @@
+// 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 fileio provides transforms for matching and reading files.
+package fileio
+
+import (
+	"context"
+	"fmt"
+	"strings"
+
+	"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"
+)
+
+func init() {
+	register.DoFn3x1[context.Context, string, func(FileMetadata), error](&matchFn{})
+	register.Emitter1[FileMetadata]()
+}
+
+// emptyTreatment controls how empty matches of a pattern are treated.
+type emptyTreatment int
+
+const (
+	// emptyAllow allows empty matches.
+	emptyAllow emptyTreatment = iota
+	// emptyDisallow disallows empty matches.
+	emptyDisallow
+	// emptyAllowIfWildcard allows empty matches if the pattern contains a wildcard.
+	emptyAllowIfWildcard
+)
+
+type matchOption struct {
+	EmptyTreatment emptyTreatment
+}
+
+// MatchOptionFn is a function that can be passed to MatchFiles or MatchAll to configure options for
+// matching files.
+type MatchOptionFn func(*matchOption)
+
+// MatchEmptyAllow specifies that empty matches are allowed.
+func MatchEmptyAllow() MatchOptionFn {
+	return func(o *matchOption) {
+		o.EmptyTreatment = emptyAllow
+	}
+}
+
+// MatchEmptyDisallow specifies that empty matches are not allowed.
+func MatchEmptyDisallow() MatchOptionFn {
+	return func(o *matchOption) {
+		o.EmptyTreatment = emptyDisallow
+	}
+}
+
+// MatchFiles finds all files matching the glob pattern and returns a PCollection<FileMetadata> of
+// the matching files. MatchFiles accepts a variadic number of MatchOptionFn that can be used to
+// configure the treatment of empty matches. By default, empty matches are allowed if the pattern
+// contains a wildcard.
+func MatchFiles(s beam.Scope, glob string, opts ...MatchOptionFn) beam.PCollection {
+	s = s.Scope("fileio.MatchFiles")
+
+	filesystem.ValidateScheme(glob)
+	return MatchAll(s, beam.Create(s, glob), opts...)
+}
+
+// MatchAll finds all files matching the glob patterns given by the incoming PCollection<string> and
+// returns a PCollection<FileMetadata> of the matching files. MatchAll accepts a variadic number of
+// MatchOptionFn that can be used to configure the treatment of empty matches. By default, empty
+// matches are allowed if the pattern contains a wildcard.
+func MatchAll(s beam.Scope, col beam.PCollection, opts ...MatchOptionFn) beam.PCollection {
+	s = s.Scope("fileio.MatchAll")
+
+	option := &matchOption{
+		EmptyTreatment: emptyAllowIfWildcard,

Review Comment:
   I haven't provided any option functions for setting what is already the default value, i.e. there is no `MatchEmptyAllowIfWildcard()`, because it doesn't make much sense? Unless it should be included for clarity/documentation purposes?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] johannaojeling commented on a diff in pull request #25809: [Go SDK]: Add fileio MatchFiles, MatchAll and ReadMatches transforms

Posted by "johannaojeling (via GitHub)" <gi...@apache.org>.
johannaojeling commented on code in PR #25809:
URL: https://github.com/apache/beam/pull/25809#discussion_r1147851156


##########
sdks/go/pkg/beam/io/fileio/match.go:
##########
@@ -0,0 +1,174 @@
+// 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 fileio provides transforms for matching and reading files.
+package fileio
+
+import (
+	"context"
+	"fmt"
+	"strings"
+
+	"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"
+)
+
+func init() {
+	register.DoFn3x1[context.Context, string, func(filesystem.FileMetadata), error](&matchFn{})
+	register.Emitter1[filesystem.FileMetadata]()
+}
+
+// EmptyMatchTreatment controls how empty matches of a pattern are treated.
+type EmptyMatchTreatment int
+
+const (
+	// EmptyMatchTreatmentAllow allows empty matches.
+	EmptyMatchTreatmentAllow EmptyMatchTreatment = iota
+	// EmptyMatchTreatmentDisallow disallows empty matches.
+	EmptyMatchTreatmentDisallow
+	// EmptyMatchTreatmentAllowIfWildcard allows empty matches if the pattern contains a wildcard.
+	EmptyMatchTreatmentAllowIfWildcard
+)
+
+type matchOption struct {
+	EmptyMatchTreatment EmptyMatchTreatment
+}
+
+// MatchOptionFn is a function that can be passed to MatchFiles or MatchAll to configure options for
+// matching files.
+type MatchOptionFn func(*matchOption) error
+
+// WithEmptyMatchTreatment specifies how empty matches of a pattern should be treated. By default,
+// empty matches are allowed if the pattern contains a wildcard.
+func WithEmptyMatchTreatment(treatment EmptyMatchTreatment) MatchOptionFn {

Review Comment:
   Apologies for the delay, it's now been updated with a suggestion. I like the way of providing the functional options without exporting the enums, it looks it simpler for the user.
   
   I do find naming of the option functions a bit tricky though, with regards to making it easy to tell which option fns are applicable to which function (for autocompletion as you mention etc.) without giving them too long names. I ended up using a naming convention with a prefix indicating which function it can be passed to (e.g. `Read`), followed by the type of configuration (`Compression`), followed by the value (`Gzip`). Do let me know if you still find it too verbose.
   
   My reason for prefixing them with `Match` and `Read` is to take into account the possibility that there could be other transforms added to the fileio package in the future, which may have some type of configuration that partly overlaps with that of other transforms. For example, what if we add a `Write` transform which can be configured with
   
   ```go
   type writeOption struct {
   	Compression compressionType
   	NumShards   int
   }
   
   type WriteOptionFn func(*writeOption)
   ```
   
   Configuring a compression is also applicable to `ReadOptionFn`, so we would need one function for each of these, i.e. `fileio.WriteCompressionGzip()` and `fileio.ReadCompressionGzip()`. I tried some solution with generics some time (similar to [this one](https://golang.design/research/generic-option/#:~:text=Using%20Generics%20(and%20Make%20Call%20Safer))), but found it cumbersome and that it put a burden on the user to select the right type.
   
   If you can think of a better way of doing it, I'm happy to change!



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] lostluck commented on a diff in pull request #25809: [Go SDK]: Add fileio MatchFiles, MatchAll and ReadMatches transforms

Posted by "lostluck (via GitHub)" <gi...@apache.org>.
lostluck commented on code in PR #25809:
URL: https://github.com/apache/beam/pull/25809#discussion_r1150980891


##########
sdks/go/pkg/beam/io/fileio/match.go:
##########
@@ -0,0 +1,181 @@
+// 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 fileio provides transforms for matching and reading files.
+package fileio
+
+import (
+	"context"
+	"fmt"
+	"strings"
+
+	"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"
+)
+
+func init() {
+	register.DoFn3x1[context.Context, string, func(FileMetadata), error](&matchFn{})
+	register.Emitter1[FileMetadata]()
+}
+
+// emptyTreatment controls how empty matches of a pattern are treated.
+type emptyTreatment int
+
+const (
+	// emptyAllow allows empty matches.
+	emptyAllow emptyTreatment = iota
+	// emptyDisallow disallows empty matches.
+	emptyDisallow
+	// emptyAllowIfWildcard allows empty matches if the pattern contains a wildcard.
+	emptyAllowIfWildcard
+)
+
+type matchOption struct {
+	EmptyTreatment emptyTreatment
+}
+
+// MatchOptionFn is a function that can be passed to MatchFiles or MatchAll to configure options for
+// matching files.
+type MatchOptionFn func(*matchOption)
+
+// MatchEmptyAllow specifies that empty matches are allowed.
+func MatchEmptyAllow() MatchOptionFn {
+	return func(o *matchOption) {
+		o.EmptyTreatment = emptyAllow
+	}
+}
+
+// MatchEmptyDisallow specifies that empty matches are not allowed.
+func MatchEmptyDisallow() MatchOptionFn {
+	return func(o *matchOption) {
+		o.EmptyTreatment = emptyDisallow
+	}
+}
+
+// MatchFiles finds all files matching the glob pattern and returns a PCollection<FileMetadata> of
+// the matching files. MatchFiles accepts a variadic number of MatchOptionFn that can be used to
+// configure the treatment of empty matches. By default, empty matches are allowed if the pattern
+// contains a wildcard.
+func MatchFiles(s beam.Scope, glob string, opts ...MatchOptionFn) beam.PCollection {
+	s = s.Scope("fileio.MatchFiles")
+
+	filesystem.ValidateScheme(glob)
+	return MatchAll(s, beam.Create(s, glob), opts...)
+}
+
+// MatchAll finds all files matching the glob patterns given by the incoming PCollection<string> and
+// returns a PCollection<FileMetadata> of the matching files. MatchAll accepts a variadic number of
+// MatchOptionFn that can be used to configure the treatment of empty matches. By default, empty
+// matches are allowed if the pattern contains a wildcard.
+func MatchAll(s beam.Scope, col beam.PCollection, opts ...MatchOptionFn) beam.PCollection {
+	s = s.Scope("fileio.MatchAll")
+
+	option := &matchOption{
+		EmptyTreatment: emptyAllowIfWildcard,

Review Comment:
   Better to have one for the default if only to explain that it's the default.
   
   Also in complex situations, it might get set multiple times in the list of options, so having the default enables setting it back into the default if desired.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] johannaojeling commented on pull request #25809: [Go SDK]: Add fileio MatchFiles, MatchAll and ReadMatches transforms

Posted by "johannaojeling (via GitHub)" <gi...@apache.org>.
johannaojeling commented on PR #25809:
URL: https://github.com/apache/beam/pull/25809#issuecomment-1489761150

   > I think I'm a touch surprised that neither of these are Splittable DoFns. I'd expect Match to be able to "sub element split on it's input globs to allow downstream processing of the files to ultimately be split down to each file (if not within the file itself), since we know/can find the size and so forth, to begin to make decent splitting decisions.
   
   Hmm I don't see clearly how the incoming glob element in matchFn should be split into restriction pairs. Only after the List operation do we know how many output elements there are per input element. I guess we could create an initial offsetrange.Restriction based on the total count of files that match the glob pattern from a List invocation, then split that into sub-ranges. The downside is that this would impose additional API calls for the same List operation in ProcessElement and require sorting of the list result to determine which files fall under the current restriction. I'm curious to hear what would be your suggestion for how to implement the Match transform with an SDF?
   
   As I saw it, the matchFn essentially serves the same purpose as expandFn currently does in textio/avroio/parquetio, plus attaches the size. I was thinking that those IOs could potentially be refactored to make use of the fileio Match and Read tranforms to reduce repetition if we want that. At least textio which would utilize both the file handle and the size from a ReadableFile.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] johannaojeling commented on a diff in pull request #25809: [Go SDK]: Add fileio MatchFiles, MatchAll and ReadMatches transforms

Posted by "johannaojeling (via GitHub)" <gi...@apache.org>.
johannaojeling commented on code in PR #25809:
URL: https://github.com/apache/beam/pull/25809#discussion_r1151078208


##########
sdks/go/pkg/beam/io/fileio/match.go:
##########
@@ -0,0 +1,181 @@
+// 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 fileio provides transforms for matching and reading files.
+package fileio
+
+import (
+	"context"
+	"fmt"
+	"strings"
+
+	"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"
+)
+
+func init() {
+	register.DoFn3x1[context.Context, string, func(FileMetadata), error](&matchFn{})
+	register.Emitter1[FileMetadata]()
+}
+
+// emptyTreatment controls how empty matches of a pattern are treated.
+type emptyTreatment int
+
+const (
+	// emptyAllow allows empty matches.
+	emptyAllow emptyTreatment = iota
+	// emptyDisallow disallows empty matches.
+	emptyDisallow
+	// emptyAllowIfWildcard allows empty matches if the pattern contains a wildcard.
+	emptyAllowIfWildcard
+)
+
+type matchOption struct {
+	EmptyTreatment emptyTreatment
+}
+
+// MatchOptionFn is a function that can be passed to MatchFiles or MatchAll to configure options for
+// matching files.
+type MatchOptionFn func(*matchOption)
+
+// MatchEmptyAllow specifies that empty matches are allowed.
+func MatchEmptyAllow() MatchOptionFn {
+	return func(o *matchOption) {
+		o.EmptyTreatment = emptyAllow
+	}
+}
+
+// MatchEmptyDisallow specifies that empty matches are not allowed.
+func MatchEmptyDisallow() MatchOptionFn {
+	return func(o *matchOption) {
+		o.EmptyTreatment = emptyDisallow
+	}
+}
+
+// MatchFiles finds all files matching the glob pattern and returns a PCollection<FileMetadata> of
+// the matching files. MatchFiles accepts a variadic number of MatchOptionFn that can be used to
+// configure the treatment of empty matches. By default, empty matches are allowed if the pattern
+// contains a wildcard.
+func MatchFiles(s beam.Scope, glob string, opts ...MatchOptionFn) beam.PCollection {
+	s = s.Scope("fileio.MatchFiles")
+
+	filesystem.ValidateScheme(glob)
+	return MatchAll(s, beam.Create(s, glob), opts...)
+}
+
+// MatchAll finds all files matching the glob patterns given by the incoming PCollection<string> and
+// returns a PCollection<FileMetadata> of the matching files. MatchAll accepts a variadic number of
+// MatchOptionFn that can be used to configure the treatment of empty matches. By default, empty
+// matches are allowed if the pattern contains a wildcard.
+func MatchAll(s beam.Scope, col beam.PCollection, opts ...MatchOptionFn) beam.PCollection {
+	s = s.Scope("fileio.MatchAll")
+
+	option := &matchOption{
+		EmptyTreatment: emptyAllowIfWildcard,

Review Comment:
   Okay, fixed now. Feel free to merge if approved



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] johannaojeling commented on pull request #25809: [Go SDK]: Add fileio MatchFiles, MatchAll and ReadMatches transforms

Posted by "johannaojeling (via GitHub)" <gi...@apache.org>.
johannaojeling commented on PR #25809:
URL: https://github.com/apache/beam/pull/25809#issuecomment-1465144203

   R: @lostluck


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] lostluck commented on a diff in pull request #25809: [Go SDK]: Add fileio MatchFiles, MatchAll and ReadMatches transforms

Posted by "lostluck (via GitHub)" <gi...@apache.org>.
lostluck commented on code in PR #25809:
URL: https://github.com/apache/beam/pull/25809#discussion_r1151000499


##########
sdks/go/pkg/beam/io/fileio/read.go:
##########
@@ -0,0 +1,126 @@
+// 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 fileio
+
+import (
+	"fmt"
+	"strings"
+
+	"github.com/apache/beam/sdks/v2/go/pkg/beam"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/register"
+)
+
+func init() {
+	register.DoFn2x1[FileMetadata, func(ReadableFile), error](&readFn{})
+	register.Emitter1[ReadableFile]()
+}
+
+// directoryTreatment controls how paths to directories are treated when reading matches.
+type directoryTreatment int
+
+const (
+	// directorySkip skips directories.
+	directorySkip directoryTreatment = iota
+	// directoryDisallow disallows directories.
+	directoryDisallow
+)
+
+type readOption struct {
+	Compression        compressionType
+	DirectoryTreatment directoryTreatment
+}
+
+// ReadOptionFn is a function that can be passed to ReadMatches to configure options for
+// reading files.
+type ReadOptionFn func(*readOption)
+
+// ReadCompressionGzip specifies that files have been compressed using gzip.
+func ReadCompressionGzip() ReadOptionFn {
+	return func(o *readOption) {
+		o.Compression = compressionGzip
+	}
+}
+
+// ReadCompressionUncompressed specifies that files have not been compressed.
+func ReadCompressionUncompressed() ReadOptionFn {

Review Comment:
   I think it's fine to not overdo the "matching prefix" thing. 
   
   `ReadGzip`
   `ReadUncompressed` are pretty clear in context for users who want those options, and in the documentation make it clear that they are compression options, and that only one can be set at a time. One thing that none of the various option function consolidations do well is making mutual exclusivity clear in it's usage pattern vs in it's documentation. An option struct has it's own benefits and issues there but can force users to have vestigial option structs...
   
   But at this stage, we should keep the functional options for consistency. Some future version of the SDK or some wrapper could do something like the [Proto MarshalOptions](https://pkg.go.dev/google.golang.org/protobuf/proto#MarshalOptions) are called out from the Options structs. But that innovation is recent WRT the SDK design...



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] johannaojeling commented on a diff in pull request #25809: [Go SDK]: Add fileio MatchFiles, MatchAll and ReadMatches transforms

Posted by "johannaojeling (via GitHub)" <gi...@apache.org>.
johannaojeling commented on code in PR #25809:
URL: https://github.com/apache/beam/pull/25809#discussion_r1142594661


##########
sdks/go/pkg/beam/io/filesystem/file.go:
##########
@@ -0,0 +1,44 @@
+// 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 filesystem
+
+import (
+	"reflect"
+
+	"github.com/apache/beam/sdks/v2/go/pkg/beam"
+)
+
+func init() {
+	beam.RegisterType(reflect.TypeOf((*FileMetadata)(nil)).Elem())
+}
+
+// FileMetadata contains metadata about a file, namely its path and size in bytes.
+type FileMetadata struct {

Review Comment:
   The equivalent types in the Java and Python SDKs are
   
   - `org.apache.beam.sdk.io.fs.MatchResult.Metadata`
   - `apache_beam.io.filesystem.FileMetadata`
   
   But I agree it is more appropriate to have it in the fileio package. Will move it!



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] johannaojeling commented on a diff in pull request #25809: [Go SDK]: Add fileio MatchFiles, MatchAll and ReadMatches transforms

Posted by "johannaojeling (via GitHub)" <gi...@apache.org>.
johannaojeling commented on code in PR #25809:
URL: https://github.com/apache/beam/pull/25809#discussion_r1147855475


##########
sdks/go/pkg/beam/io/fileio/read.go:
##########
@@ -0,0 +1,126 @@
+// 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 fileio
+
+import (
+	"fmt"
+	"strings"
+
+	"github.com/apache/beam/sdks/v2/go/pkg/beam"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/register"
+)
+
+func init() {
+	register.DoFn2x1[FileMetadata, func(ReadableFile), error](&readFn{})
+	register.Emitter1[ReadableFile]()
+}
+
+// directoryTreatment controls how paths to directories are treated when reading matches.
+type directoryTreatment int
+
+const (
+	// directorySkip skips directories.
+	directorySkip directoryTreatment = iota
+	// directoryDisallow disallows directories.
+	directoryDisallow
+)
+
+type readOption struct {
+	Compression        compressionType
+	DirectoryTreatment directoryTreatment
+}
+
+// ReadOptionFn is a function that can be passed to ReadMatches to configure options for
+// reading files.
+type ReadOptionFn func(*readOption)
+
+// ReadCompressionGzip specifies that files have been compressed using gzip.
+func ReadCompressionGzip() ReadOptionFn {
+	return func(o *readOption) {
+		o.Compression = compressionGzip
+	}
+}
+
+// ReadCompressionUncompressed specifies that files have not been compressed.
+func ReadCompressionUncompressed() ReadOptionFn {

Review Comment:
   I think this still looks a bit ugly but I can't come up with a good short name. Only `ReadNoCompression` or `ReadWithoutCompression` which potentially read better, but then the functions would no longer have the shared prefix...



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] lostluck commented on a diff in pull request #25809: [Go SDK]: Add fileio MatchFiles, MatchAll and ReadMatches transforms

Posted by "lostluck (via GitHub)" <gi...@apache.org>.
lostluck commented on code in PR #25809:
URL: https://github.com/apache/beam/pull/25809#discussion_r1152452225


##########
sdks/go/pkg/beam/io/fileio/gzip.go:
##########
@@ -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 fileio
+
+import (
+	"compress/gzip"
+	"context"
+	"io"
+
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/log"
+)
+
+// gzipReader is a wrapper around a gzip.Reader that also closes the underlying io.ReadCloser.
+type gzipReader struct {
+	rc io.ReadCloser
+	zr *gzip.Reader
+}
+
+// newGzipReader creates a new gzipReader from an io.ReadCloser.
+func newGzipReader(rc io.ReadCloser) (*gzipReader, error) {

Review Comment:
   I think longer term when if add additional compression formats, we'd want to do something similar to how the Filesystems are set up, so users don't need to have *all possible compressions* linked in. 
   
   I think it's fine to have Gzip built in by default as it's most common, but if we build in support for more, it should be made modular, and hooked into the "auto" support.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] lostluck commented on a diff in pull request #25809: [Go SDK]: Add fileio MatchFiles, MatchAll and ReadMatches transforms

Posted by "lostluck (via GitHub)" <gi...@apache.org>.
lostluck commented on code in PR #25809:
URL: https://github.com/apache/beam/pull/25809#discussion_r1141405566


##########
sdks/go/pkg/beam/io/filesystem/file.go:
##########
@@ -0,0 +1,44 @@
+// 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 filesystem
+
+import (
+	"reflect"
+
+	"github.com/apache/beam/sdks/v2/go/pkg/beam"
+)
+
+func init() {
+	beam.RegisterType(reflect.TypeOf((*FileMetadata)(nil)).Elem())
+}
+
+// FileMetadata contains metadata about a file, namely its path and size in bytes.
+type FileMetadata struct {

Review Comment:
   Is the longer term goal to have FileMetadata manipulated by the filesystem abstractions? If not, we should just move this to the `fileio` package. It's not typical in Go to define a type that's not used within the package itself. Exported types and fields are a package's API, and in this case, it's not controlling anything here.
   
   Is there a pattern from Java or Python being replicated?



##########
sdks/go/pkg/beam/io/fileio/match.go:
##########
@@ -0,0 +1,174 @@
+// 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 fileio provides transforms for matching and reading files.
+package fileio
+
+import (
+	"context"
+	"fmt"
+	"strings"
+
+	"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"
+)
+
+func init() {
+	register.DoFn3x1[context.Context, string, func(filesystem.FileMetadata), error](&matchFn{})
+	register.Emitter1[filesystem.FileMetadata]()
+}
+
+// EmptyMatchTreatment controls how empty matches of a pattern are treated.
+type EmptyMatchTreatment int
+
+const (
+	// EmptyMatchTreatmentAllow allows empty matches.
+	EmptyMatchTreatmentAllow EmptyMatchTreatment = iota
+	// EmptyMatchTreatmentDisallow disallows empty matches.
+	EmptyMatchTreatmentDisallow
+	// EmptyMatchTreatmentAllowIfWildcard allows empty matches if the pattern contains a wildcard.
+	EmptyMatchTreatmentAllowIfWildcard
+)
+
+type matchOption struct {
+	EmptyMatchTreatment EmptyMatchTreatment
+}
+
+// MatchOptionFn is a function that can be passed to MatchFiles or MatchAll to configure options for
+// matching files.
+type MatchOptionFn func(*matchOption) error
+
+// WithEmptyMatchTreatment specifies how empty matches of a pattern should be treated. By default,
+// empty matches are allowed if the pattern contains a wildcard.
+func WithEmptyMatchTreatment(treatment EmptyMatchTreatment) MatchOptionFn {

Review Comment:
   This applies to the ReadOptions and treatments as well.
   
   A user passes the option in like `fileio.WithEmptyMatchTreatment(fileio.EmptyMatchTreatmentAllowIfWildcard)` and is 73 characters by itself. That's very long, and repeats itself several times.
   
   And then we have all the options with field names matching the types directly. What if instead of having an exported enum consts we just have the set of helper functions directly?
   
   Having users instead call `fileio.EmptyMatchAllowedIfWildcard` remains as clear, avoids extra structural words like "With" and "Treatment", keeps a common prefix for keeping related options together in autocompletes.
   
   The underlying mechanism can still be managed with an enum and such, but then we don't expose unnecessary implementation details to users. This also reduces the odds of users passing in nonsense values for the enum, since when the `EmptyMatchTreatment` is exported, someone can cast that type. It's unlikely for this to happen, but why give them the option? Restricting the space of possibility makes APIs clearer to understand and use, since there are fewer opportunities for misuse.
   
   One could argue this makes it impossible for users to provide their own options... but the function that takes constant doesn't enable this, since a developer would still need to add new code to the package anyway. For end user packages, it's the end user's writing experience that needs to be optimized not the package's author (AKA us). And this just means adding 3 more lines (the function call), and moves the documentation directly anyway.
   
   ---------
   
   Two implementation options: Simple function pointers (AKA the function is the option func directly)
   
   ```
   func EmptyMatchAllowedIfWildcard(mo *matchOption) error {
      o.EmptyMatchTreatment = emptyMatchTreatmentAllowIfWildcard
   }
   ```
   
   or more similar to the current implementation, returning the option fn.
   
   ```
   func EmptyMatchAllowedIfWildcard() MatchOptionFn {
   	return func(o *matchOption) error {
   		o.EmptyMatchTreatment = emptyMatchTreatmentAllowIfWildcard
   		return nil
   	}
   }
   ```
   
   Or the same with a helper that's the same as the current `WithEmptyMatchTreatment`.
   
   ```
   func EmptyMatchAllowedIfWildcard() MatchOptionFn {
   	return withEmptyMatchTreatment(emptyMatchTreatmentAllowIfWildcard)
   		return nil
   	}
   }
   ```
   
   
   Personally, I think in this case, the former is preferred since it's just unnecessary indirection. It avoids two characters. 
   
   On the other hand, a gold standard in managing optional configuration is the `cmp` package, and it's tag along package `cmpopts`. Those defined their Options as an interface type with no exported methods (but one [internal filter method](https://github.com/google/go-cmp/blob/v0.5.9/cmp/options.go#L25)). So the lesson here is it's probably better to have users call a function that returns the option type, instead of it being direct, if only to better match user expectations around this sort of thing.
   
   So I'd say have users call functions to get the option func, if only to enable unforeseen changes in the future.
   
   Now I want to re-write the PAssert package with that in mind.... 
   
   ----
   
   Finally, since we end up hiding the enums for internal use only, we can reduce their names down. Eg. `emptyMatchTreatmentAllowIfWildcard` can be `emtAllowIfWildcard` (The type name can remain `emptyMatchTreatment`., and same for `directoryTreatment` with `dt` prefixes.
   
   Compression is the odd one out here (no good feelings on this), but if we can't think of a good reason for it not to follow the same pattern, we should be consistent with it.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] johannaojeling commented on a diff in pull request #25809: [Go SDK]: Add fileio MatchFiles, MatchAll and ReadMatches transforms

Posted by "johannaojeling (via GitHub)" <gi...@apache.org>.
johannaojeling commented on code in PR #25809:
URL: https://github.com/apache/beam/pull/25809#discussion_r1142595707


##########
sdks/go/pkg/beam/io/fileio/match.go:
##########
@@ -0,0 +1,174 @@
+// 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 fileio provides transforms for matching and reading files.
+package fileio
+
+import (
+	"context"
+	"fmt"
+	"strings"
+
+	"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"
+)
+
+func init() {
+	register.DoFn3x1[context.Context, string, func(filesystem.FileMetadata), error](&matchFn{})
+	register.Emitter1[filesystem.FileMetadata]()
+}
+
+// EmptyMatchTreatment controls how empty matches of a pattern are treated.
+type EmptyMatchTreatment int
+
+const (
+	// EmptyMatchTreatmentAllow allows empty matches.
+	EmptyMatchTreatmentAllow EmptyMatchTreatment = iota
+	// EmptyMatchTreatmentDisallow disallows empty matches.
+	EmptyMatchTreatmentDisallow
+	// EmptyMatchTreatmentAllowIfWildcard allows empty matches if the pattern contains a wildcard.
+	EmptyMatchTreatmentAllowIfWildcard
+)
+
+type matchOption struct {
+	EmptyMatchTreatment EmptyMatchTreatment
+}
+
+// MatchOptionFn is a function that can be passed to MatchFiles or MatchAll to configure options for
+// matching files.
+type MatchOptionFn func(*matchOption) error
+
+// WithEmptyMatchTreatment specifies how empty matches of a pattern should be treated. By default,
+// empty matches are allowed if the pattern contains a wildcard.
+func WithEmptyMatchTreatment(treatment EmptyMatchTreatment) MatchOptionFn {

Review Comment:
   Thank you so much for your input! I didn't like how verbose the code became either but didn't think of a better way to do it. Your suggestions are much nicer. I'll update the PR during the week. Thanks!



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] lostluck commented on a diff in pull request #25809: [Go SDK]: Add fileio MatchFiles, MatchAll and ReadMatches transforms

Posted by "lostluck (via GitHub)" <gi...@apache.org>.
lostluck commented on code in PR #25809:
URL: https://github.com/apache/beam/pull/25809#discussion_r1141400660


##########
sdks/go/pkg/beam/io/filesystem/file.go:
##########
@@ -0,0 +1,44 @@
+// 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 filesystem

Review Comment:
   I vote it should be part of the `fileio` package, rather than the `filesystem` package. Filesystems, if they're compressed, would be abstracted away when reading from them, while files themselves could be compressed, and it's necessary to know that in order to read them.
   
   I see Python puts it in the filesystem code but it feels tacked on, leading to [repetition for each file system](https://github.com/apache/beam/search?l=Python&q=_get_compression_type), rather than isolating that decision to a general fileio. 
   
   Java keeps it in it's [own class](https://github.com/apache/beam/blob/48bad7d966a583055669850eb9fb558782f636a8/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Compression.java) in the general IOs, and it's largely invoking the CompressedSource or similar delegating to the single implementation around files.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] johannaojeling commented on a diff in pull request #25809: [Go SDK]: Add fileio MatchFiles, MatchAll and ReadMatches transforms

Posted by "johannaojeling (via GitHub)" <gi...@apache.org>.
johannaojeling commented on code in PR #25809:
URL: https://github.com/apache/beam/pull/25809#discussion_r1142596384


##########
sdks/go/pkg/beam/io/fileio/match.go:
##########
@@ -0,0 +1,174 @@
+// 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 fileio provides transforms for matching and reading files.
+package fileio
+
+import (
+	"context"
+	"fmt"
+	"strings"
+
+	"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"
+)
+
+func init() {
+	register.DoFn3x1[context.Context, string, func(filesystem.FileMetadata), error](&matchFn{})
+	register.Emitter1[filesystem.FileMetadata]()
+}
+
+// EmptyMatchTreatment controls how empty matches of a pattern are treated.
+type EmptyMatchTreatment int
+
+const (
+	// EmptyMatchTreatmentAllow allows empty matches.
+	EmptyMatchTreatmentAllow EmptyMatchTreatment = iota
+	// EmptyMatchTreatmentDisallow disallows empty matches.
+	EmptyMatchTreatmentDisallow
+	// EmptyMatchTreatmentAllowIfWildcard allows empty matches if the pattern contains a wildcard.
+	EmptyMatchTreatmentAllowIfWildcard
+)
+
+type matchOption struct {
+	EmptyMatchTreatment EmptyMatchTreatment
+}
+
+// MatchOptionFn is a function that can be passed to MatchFiles or MatchAll to configure options for
+// matching files.
+type MatchOptionFn func(*matchOption) error
+
+// WithEmptyMatchTreatment specifies how empty matches of a pattern should be treated. By default,
+// empty matches are allowed if the pattern contains a wildcard.
+func WithEmptyMatchTreatment(treatment EmptyMatchTreatment) MatchOptionFn {

Review Comment:
   Also, I have to say, it's so convenient to test pipelines with prism and get feedback instantly, instead of having to wait for a Dataflow/Flink/Spark job.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] lostluck merged pull request #25809: [Go SDK]: Add fileio MatchFiles, MatchAll and ReadMatches transforms

Posted by "lostluck (via GitHub)" <gi...@apache.org>.
lostluck merged PR #25809:
URL: https://github.com/apache/beam/pull/25809


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] johannaojeling commented on a diff in pull request #25809: [Go SDK]: Add fileio MatchFiles, MatchAll and ReadMatches transforms

Posted by "johannaojeling (via GitHub)" <gi...@apache.org>.
johannaojeling commented on code in PR #25809:
URL: https://github.com/apache/beam/pull/25809#discussion_r1148563739


##########
sdks/go/pkg/beam/io/fileio/match.go:
##########
@@ -0,0 +1,174 @@
+// 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 fileio provides transforms for matching and reading files.
+package fileio
+
+import (
+	"context"
+	"fmt"
+	"strings"
+
+	"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"
+)
+
+func init() {
+	register.DoFn3x1[context.Context, string, func(filesystem.FileMetadata), error](&matchFn{})
+	register.Emitter1[filesystem.FileMetadata]()
+}
+
+// EmptyMatchTreatment controls how empty matches of a pattern are treated.
+type EmptyMatchTreatment int
+
+const (
+	// EmptyMatchTreatmentAllow allows empty matches.
+	EmptyMatchTreatmentAllow EmptyMatchTreatment = iota
+	// EmptyMatchTreatmentDisallow disallows empty matches.
+	EmptyMatchTreatmentDisallow
+	// EmptyMatchTreatmentAllowIfWildcard allows empty matches if the pattern contains a wildcard.
+	EmptyMatchTreatmentAllowIfWildcard
+)
+
+type matchOption struct {
+	EmptyMatchTreatment EmptyMatchTreatment
+}
+
+// MatchOptionFn is a function that can be passed to MatchFiles or MatchAll to configure options for
+// matching files.
+type MatchOptionFn func(*matchOption) error
+
+// WithEmptyMatchTreatment specifies how empty matches of a pattern should be treated. By default,
+// empty matches are allowed if the pattern contains a wildcard.
+func WithEmptyMatchTreatment(treatment EmptyMatchTreatment) MatchOptionFn {

Review Comment:
   For reference, if using the current approach, my suggested implementation for support reading gzip compressed files in textio would probably look something like [a540fab](https://github.com/johannaojeling/beam/commit/a540fab6b5cc88a55a0a3e3f4c13fd9c2ad6d15f)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] codecov[bot] commented on pull request #25809: [Go SDK]: Add fileio MatchFiles, MatchAll and ReadMatches transforms

Posted by "codecov[bot] (via GitHub)" <gi...@apache.org>.
codecov[bot] commented on PR #25809:
URL: https://github.com/apache/beam/pull/25809#issuecomment-1465145494

   ## [Codecov](https://codecov.io/gh/apache/beam/pull/25809?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#25809](https://codecov.io/gh/apache/beam/pull/25809?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (416df16) into [master](https://codecov.io/gh/apache/beam/commit/17f7a5d1aff227d211c634492f4396c8422678b5?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (17f7a5d) will **increase** coverage by `0.01%`.
   > The diff coverage is `79.60%`.
   
   ```diff
   @@            Coverage Diff             @@
   ##           master   #25809      +/-   ##
   ==========================================
   + Coverage   72.73%   72.75%   +0.01%     
   ==========================================
     Files         777      782       +5     
     Lines      103074   103275     +201     
   ==========================================
   + Hits        74971    75134     +163     
   - Misses      26648    26674      +26     
   - Partials     1455     1467      +12     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | go | `54.12% <79.60%> (+0.16%)` | :arrow_up: |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/beam/pull/25809?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [sdks/go/pkg/beam/io/fileio/gzip.go](https://codecov.io/gh/apache/beam/pull/25809?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2Rrcy9nby9wa2cvYmVhbS9pby9maWxlaW8vZ3ppcC5nbw==) | `55.55% <55.55%> (ø)` | |
   | [sdks/go/pkg/beam/io/fileio/file.go](https://codecov.io/gh/apache/beam/pull/25809?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2Rrcy9nby9wa2cvYmVhbS9pby9maWxlaW8vZmlsZS5nbw==) | `68.51% <68.51%> (ø)` | |
   | [sdks/go/pkg/beam/io/fileio/match.go](https://codecov.io/gh/apache/beam/pull/25809?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2Rrcy9nby9wa2cvYmVhbS9pby9maWxlaW8vbWF0Y2guZ28=) | `82.27% <82.27%> (ø)` | |
   | [sdks/go/pkg/beam/io/fileio/read.go](https://codecov.io/gh/apache/beam/pull/25809?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2Rrcy9nby9wa2cvYmVhbS9pby9maWxlaW8vcmVhZC5nbw==) | `95.83% <95.83%> (ø)` | |
   | [sdks/go/pkg/beam/io/filesystem/file.go](https://codecov.io/gh/apache/beam/pull/25809?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2Rrcy9nby9wa2cvYmVhbS9pby9maWxlc3lzdGVtL2ZpbGUuZ28=) | `100.00% <100.00%> (ø)` | |
   
   ... and [1 file with indirect coverage changes](https://codecov.io/gh/apache/beam/pull/25809/indirect-changes?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   :mega: We’re building smart automated test selection to slash your CI/CD build times. [Learn more](https://about.codecov.io/iterative-testing/?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] johannaojeling commented on a diff in pull request #25809: [Go SDK]: Add fileio MatchFiles, MatchAll and ReadMatches transforms

Posted by "johannaojeling (via GitHub)" <gi...@apache.org>.
johannaojeling commented on code in PR #25809:
URL: https://github.com/apache/beam/pull/25809#discussion_r1133222795


##########
sdks/go/pkg/beam/io/filesystem/file.go:
##########
@@ -0,0 +1,44 @@
+// 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 filesystem

Review Comment:
   I was a bit undecided about whether to have the `Compression` and `FileMetadata` types defined in the `filesystem` or `fileio` package. They are currently only used in fileio, however having them in filesystem appears more in line with the other SDKs. I also want to use the Compression type in textio (#25780), leading to user code like:
   
   ```go
   lines := textio.Read(s, glob, textio.WithReadCompression(filesystem.CompressionGzip))
   ```
   
   vs
   
   ```go
   lines := textio.Read(s, glob, textio.WithReadCompression(fileio.CompressionGzip))
   ```
   
   What is your opinion?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] github-actions[bot] commented on pull request #25809: [Go SDK]: Add fileio MatchFiles, MatchAll and ReadMatches transforms

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] commented on PR #25809:
URL: https://github.com/apache/beam/pull/25809#issuecomment-1465144455

   Stopping reviewer notifications for this pull request: review requested by someone other than the bot, ceding control


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org