/
match.go
189 lines (157 loc) · 4.83 KB
/
match.go
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
// 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)
// MatchEmptyAllowIfWildcard specifies that empty matches are allowed if the pattern contains a
// wildcard.
func MatchEmptyAllowIfWildcard() MatchOptionFn {
return func(o *matchOption) {
o.EmptyTreatment = emptyAllowIfWildcard
}
}
// 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,
}
for _, opt := range opts {
opt(option)
}
return beam.ParDo(s, newMatchFn(option), col)
}
type matchFn struct {
EmptyTreatment emptyTreatment
}
func newMatchFn(option *matchOption) *matchFn {
return &matchFn{
EmptyTreatment: option.EmptyTreatment,
}
}
func (fn *matchFn) ProcessElement(
ctx context.Context,
glob string,
emit func(FileMetadata),
) error {
if strings.TrimSpace(glob) == "" {
return nil
}
fs, err := filesystem.New(ctx, glob)
if err != nil {
return err
}
defer fs.Close()
files, err := fs.List(ctx, glob)
if err != nil {
return err
}
if len(files) == 0 {
if !allowEmptyMatch(glob, fn.EmptyTreatment) {
return fmt.Errorf("no files matching pattern %q", glob)
}
return nil
}
metadata, err := metadataFromFiles(ctx, fs, files)
if err != nil {
return err
}
for _, md := range metadata {
emit(md)
}
return nil
}
func allowEmptyMatch(glob string, treatment emptyTreatment) bool {
switch treatment {
case emptyDisallow:
return false
case emptyAllowIfWildcard:
return strings.Contains(glob, "*")
default:
return true
}
}
func metadataFromFiles(
ctx context.Context,
fs filesystem.Interface,
files []string,
) ([]FileMetadata, error) {
if len(files) == 0 {
return nil, nil
}
metadata := make([]FileMetadata, len(files))
for i, path := range files {
size, err := fs.Size(ctx, path)
if err != nil {
return nil, err
}
metadata[i] = FileMetadata{
Path: path,
Size: size,
}
}
return metadata, nil
}