/
textio.go
180 lines (148 loc) · 4.74 KB
/
textio.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
// 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 textio contains transforms for reading and writing text files.
package textio
import (
"bufio"
"context"
"os"
"reflect"
"strings"
"github.com/apache/beam/sdks/go/pkg/beam"
"github.com/apache/beam/sdks/go/pkg/beam/io/filesystem"
"github.com/apache/beam/sdks/go/pkg/beam/log"
)
func init() {
beam.RegisterType(reflect.TypeOf((*writeFileFn)(nil)).Elem())
beam.RegisterFunction(readFn)
beam.RegisterFunction(expandFn)
}
// Read reads a set of file and returns the lines as a PCollection<string>. The
// newlines are not part of the lines.
func Read(s beam.Scope, glob string) beam.PCollection {
s = s.Scope("textio.Read")
filesystem.ValidateScheme(glob)
return read(s, beam.Create(s, glob))
}
// ReadAll expands and reads the filename given as globs by the incoming
// PCollection<string>. It returns the lines of all files as a single
// PCollection<string>. The newlines are not part of the lines.
func ReadAll(s beam.Scope, col beam.PCollection) beam.PCollection {
s = s.Scope("textio.ReadAll")
return read(s, col)
}
func read(s beam.Scope, col beam.PCollection) beam.PCollection {
files := beam.ParDo(s, expandFn, col)
return beam.ParDo(s, readFn, files)
}
func expandFn(ctx context.Context, glob string, emit func(string)) error {
if strings.TrimSpace(glob) == "" {
return nil // ignore empty string elements here
}
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
}
for _, filename := range files {
emit(filename)
}
return nil
}
func readFn(ctx context.Context, filename string, emit func(string)) error {
log.Infof(ctx, "Reading from %v", filename)
fs, err := filesystem.New(ctx, filename)
if err != nil {
return err
}
defer fs.Close()
fd, err := fs.OpenRead(ctx, filename)
if err != nil {
return err
}
defer fd.Close()
scanner := bufio.NewScanner(fd)
for scanner.Scan() {
emit(scanner.Text())
}
return scanner.Err()
}
// TODO(herohde) 7/12/2017: extend Write to write to a series of files
// as well as allow sharding.
// Write writes a PCollection<string> to a file as separate lines. The
// writer add a newline after each element.
func Write(s beam.Scope, filename string, col beam.PCollection) {
s = s.Scope("textio.Write")
filesystem.ValidateScheme(filename)
// NOTE(BEAM-3579): We may never call Teardown for non-local runners and
// FinishBundle doesn't have the right granularity. We therefore
// perform a GBK with a fixed key to get all values in a single invocation.
// TODO(BEAM-3860) 3/15/2018: use side input instead of GBK.
pre := beam.AddFixedKey(s, col)
post := beam.GroupByKey(s, pre)
beam.ParDo0(s, &writeFileFn{Filename: filename}, post)
}
type writeFileFn struct {
Filename string `json:"filename"`
}
func (w *writeFileFn) ProcessElement(ctx context.Context, _ int, lines func(*string) bool) error {
fs, err := filesystem.New(ctx, w.Filename)
if err != nil {
return err
}
defer fs.Close()
fd, err := fs.OpenWrite(ctx, w.Filename)
if err != nil {
return err
}
buf := bufio.NewWriterSize(fd, 1<<20) // use 1MB buffer
log.Infof(ctx, "Writing to %v", w.Filename)
var line string
for lines(&line) {
if _, err := buf.WriteString(line); err != nil {
return err
}
if _, err := buf.Write([]byte{'\n'}); err != nil {
return err
}
}
if err := buf.Flush(); err != nil {
return err
}
return fd.Close()
}
// Immediate reads a local file at pipeline construction-time and embeds the
// data into a I/O-free pipeline source. Should be used for small files only.
func Immediate(s beam.Scope, filename string) (beam.PCollection, error) {
s = s.Scope("textio.Immediate")
var data []interface{}
file, err := os.Open(filename)
if err != nil {
return beam.PCollection{}, err
}
defer file.Close()
scanner := bufio.NewScanner(file)
for scanner.Scan() {
data = append(data, scanner.Text())
}
if err := scanner.Err(); err != nil {
return beam.PCollection{}, err
}
return beam.Create(s, data...), nil
}