-
Notifications
You must be signed in to change notification settings - Fork 4.5k
Expand file tree
/
Copy pathbatch_dofn_test.py
More file actions
301 lines (236 loc) · 10.2 KB
/
batch_dofn_test.py
File metadata and controls
301 lines (236 loc) · 10.2 KB
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
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
#
# 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.
#
"""UnitTests for Batched DoFn (process_batch) API."""
# pytype: skip-file
import unittest
from collections.abc import Iterator
from typing import no_type_check
from parameterized import parameterized_class
import apache_beam as beam
class ElementDoFn(beam.DoFn):
def process(self, element: int, *args, **kwargs) -> Iterator[float]:
yield element / 2
class BatchDoFn(beam.DoFn):
def process_batch(self, batch: list[int], *args,
**kwargs) -> Iterator[list[float]]:
yield [element / 2 for element in batch]
class NoReturnAnnotation(beam.DoFn):
def process_batch(self, batch: list[int], *args, **kwargs):
yield [element * 2 for element in batch]
class OverrideTypeInference(beam.DoFn):
def process_batch(self, batch, *args, **kwargs):
yield [element * 2 for element in batch]
def get_input_batch_type(self, input_element_type):
return list[input_element_type]
def get_output_batch_type(self, input_element_type):
return list[input_element_type]
class EitherDoFn(beam.DoFn):
def process(self, element: int, *args, **kwargs) -> Iterator[float]:
yield element / 2
def process_batch(self, batch: list[int], *args,
**kwargs) -> Iterator[list[float]]:
yield [element / 2 for element in batch]
class ElementToBatchDoFn(beam.DoFn):
@beam.DoFn.yields_batches
def process(self, element: int, *args, **kwargs) -> Iterator[list[int]]:
yield [element] * element
def infer_output_type(self, input_element_type):
return input_element_type
class BatchToElementDoFn(beam.DoFn):
@beam.DoFn.yields_elements
def process_batch(self, batch: list[int], *args,
**kwargs) -> Iterator[tuple[int, int]]:
yield (sum(batch), len(batch))
def get_test_class_name(cls, num, params_dict):
return "%s_%s" % (cls.__name__, params_dict['dofn'].__class__.__name__)
@parameterized_class([
{
"dofn": ElementDoFn(),
"input_element_type": int,
"expected_process_defined": True,
"expected_process_batch_defined": False,
"expected_input_batch_type": None,
"expected_output_batch_type": None
},
{
"dofn": BatchDoFn(),
"input_element_type": int,
"expected_process_defined": False,
"expected_process_batch_defined": True,
"expected_input_batch_type": beam.typehints.List[int],
"expected_output_batch_type": beam.typehints.List[float]
},
{
"dofn": NoReturnAnnotation(),
"input_element_type": int,
"expected_process_defined": False,
"expected_process_batch_defined": True,
"expected_input_batch_type": beam.typehints.List[int],
"expected_output_batch_type": beam.typehints.List[int]
},
{
"dofn": OverrideTypeInference(),
"input_element_type": int,
"expected_process_defined": False,
"expected_process_batch_defined": True,
"expected_input_batch_type": beam.typehints.List[int],
"expected_output_batch_type": beam.typehints.List[int]
},
{
"dofn": EitherDoFn(),
"input_element_type": int,
"expected_process_defined": True,
"expected_process_batch_defined": True,
"expected_input_batch_type": beam.typehints.List[int],
"expected_output_batch_type": beam.typehints.List[float]
},
{
"dofn": ElementToBatchDoFn(),
"input_element_type": int,
"expected_process_defined": True,
"expected_process_batch_defined": False,
"expected_input_batch_type": None,
"expected_output_batch_type": beam.typehints.List[int]
},
{
"dofn": BatchToElementDoFn(),
"input_element_type": int,
"expected_process_defined": False,
"expected_process_batch_defined": True,
"expected_input_batch_type": beam.typehints.List[int],
"expected_output_batch_type": None,
},
],
class_name_func=get_test_class_name)
class BatchDoFnParameterizedTest(unittest.TestCase):
def test_process_defined(self):
self.assertEqual(self.dofn._process_defined, self.expected_process_defined)
def test_process_batch_defined(self):
self.assertEqual(
self.dofn._process_batch_defined, self.expected_process_batch_defined)
def test_get_input_batch_type(self):
self.assertEqual(
self.dofn._get_input_batch_type_normalized(self.input_element_type),
self.expected_input_batch_type)
def test_get_output_batch_type(self):
self.assertEqual(
self.dofn._get_output_batch_type_normalized(self.input_element_type),
self.expected_output_batch_type)
def test_can_yield_batches(self):
expected = self.expected_output_batch_type is not None
self.assertEqual(self.dofn._can_yield_batches, expected)
class NoInputAnnotation(beam.DoFn):
def process_batch(self, batch, *args, **kwargs):
yield [element * 2 for element in batch]
class MismatchedBatchProducingDoFn(beam.DoFn):
"""A DoFn that produces batches from both process and process_batch, with
mismatched return types (one yields floats, the other ints). Should yield
a construction time error when applied."""
@beam.DoFn.yields_batches
def process(self, element: int, *args, **kwargs) -> Iterator[list[int]]:
yield [element]
def process_batch(self, batch: list[int], *args,
**kwargs) -> Iterator[list[float]]:
yield [element / 2 for element in batch]
class MismatchedElementProducingDoFn(beam.DoFn):
"""A DoFn that produces elements from both process and process_batch, with
mismatched return types (one yields floats, the other ints). Should yield
a construction time error when applied."""
def process(self, element: int, *args, **kwargs) -> Iterator[float]:
yield element / 2
@beam.DoFn.yields_elements
def process_batch(self, batch: list[int], *args, **kwargs) -> Iterator[int]:
yield batch[0]
class NoElementOutputAnnotation(beam.DoFn):
def process_batch(self, batch: list[int], *args,
**kwargs) -> Iterator[list[int]]:
yield [element * 2 for element in batch]
class BatchDoFnTest(unittest.TestCase):
def test_map_pardo(self):
# verify batch dofn accessors work well with beam.Map generated DoFn
# checking this in parameterized test causes a circular reference issue
dofn = beam.Map(lambda x: x * 2).dofn
self.assertTrue(dofn._process_defined)
self.assertFalse(dofn._process_batch_defined)
self.assertEqual(dofn._get_input_batch_type_normalized(int), None)
self.assertEqual(dofn._get_output_batch_type_normalized(int), None)
def test_no_input_annotation_raises(self):
p = beam.Pipeline()
pc = p | beam.Create([1, 2, 3])
with self.assertRaisesRegex(TypeError, r'NoInputAnnotation.process_batch'):
_ = pc | beam.ParDo(NoInputAnnotation())
def test_unsupported_dofn_param_raises(self):
class BadParam(beam.DoFn):
@no_type_check
def process_batch(self, batch: list[int], key=beam.DoFn.KeyParam):
yield batch * key
p = beam.Pipeline()
pc = p | beam.Create([1, 2, 3])
with self.assertRaisesRegex(NotImplementedError, r'BadParam.*KeyParam'):
_ = pc | beam.ParDo(BadParam())
def test_mismatched_batch_producer_raises(self):
p = beam.Pipeline()
pc = p | beam.Create([1, 2, 3])
# Note (?ms) makes this a multiline regex, where . matches newlines.
# See (?aiLmsux) at
# https://docs.python.org/3.4/library/re.html#regular-expression-syntax
with self.assertRaisesRegex(
TypeError,
(r'(?ms)MismatchedBatchProducingDoFn.*'
r'process: List\[<class \'int\'>\].*process_batch: '
r'List\[<class \'float\'>\]')):
_ = pc | beam.ParDo(MismatchedBatchProducingDoFn())
def test_mismatched_element_producer_raises(self):
p = beam.Pipeline()
pc = p | beam.Create([1, 2, 3])
# Note (?ms) makes this a multiline regex, where . matches newlines.
# See (?aiLmsux) at
# https://docs.python.org/3.4/library/re.html#regular-expression-syntax
with self.assertRaisesRegex(
TypeError,
r'(?ms)MismatchedElementProducingDoFn.*process:.*process_batch:'):
_ = pc | beam.ParDo(MismatchedElementProducingDoFn())
def test_cant_infer_batchconverter_input_raises(self):
p = beam.Pipeline()
pc = p | beam.Create(['a', 'b', 'c'])
with self.assertRaisesRegex(
TypeError, # Error should mention "input", and the name of the DoFn
r'input.*BatchDoFn.*'):
_ = pc | beam.ParDo(BatchDoFn())
def test_cant_infer_batchconverter_output_raises(self):
p = beam.Pipeline()
pc = p | beam.Create([1, 2, 3])
with self.assertRaisesRegex(
TypeError,
# Error should mention "output", the name of the DoFn, and suggest
# overriding DoFn.infer_output_type
r'output.*NoElementOutputAnnotation.*DoFn\.infer_output_type'):
_ = pc | beam.ParDo(NoElementOutputAnnotation())
def test_element_to_batch_dofn_typehint(self):
# Verify that element to batch DoFn sets the correct typehint on the output
# PCollection.
p = beam.Pipeline()
pc = (p | beam.Create([1, 2, 3]) | beam.ParDo(ElementToBatchDoFn()))
self.assertEqual(pc.element_type, int)
def test_batch_to_element_dofn_typehint(self):
# Verify that batch to element DoFn sets the correct typehint on the output
# PCollection.
p = beam.Pipeline()
pc = (p | beam.Create([1, 2, 3]) | beam.ParDo(BatchToElementDoFn()))
self.assertEqual(pc.element_type, beam.typehints.Tuple[int, int])
if __name__ == '__main__':
unittest.main()