-
Notifications
You must be signed in to change notification settings - Fork 4.5k
Expand file tree
/
Copy pathprogramming_guide_test.py
More file actions
424 lines (389 loc) · 13.3 KB
/
programming_guide_test.py
File metadata and controls
424 lines (389 loc) · 13.3 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
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
#
# 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.
#
import logging
import unittest
import apache_beam as beam
from apache_beam.testing.util import assert_that
from apache_beam.testing.util import equal_to
from apache_beam.yaml.readme_test import create_test_method
from apache_beam.yaml.yaml_transform import YamlTransform
DATA = [
beam.Row(animal='cat', weight=1),
beam.Row(animal='cat', weight=5),
beam.Row(animal='dog', weight=10),
]
class ProgrammingGuideTest(unittest.TestCase):
test_pipelines_constructing_reading = create_test_method(
'RUN',
'test_pipelines_constructing_reading',
'''
# [START pipelines_constructing_reading]
pipeline:
source:
type: ReadFromText
config:
path: ...
# [END pipelines_constructing_reading]
transforms: []
''')
test_create_pcollection = create_test_method(
'RUN',
'test_create_pcollection',
'''
# [START create_pcollection]
pipeline:
transforms:
- type: Create
config:
elements:
- A
- B
- ...
# [END create_pcollection]
transforms: []
''')
def test_group_by(self):
with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions(
pickle_library='cloudpickle')) as p:
elements = p | beam.Create(DATA)
result = elements | YamlTransform(
'''
# [START group_by]
type: Combine
config:
group_by: animal
combine:
weight: group
# [END group_by]
''')
assert_that(
result | beam.Map(lambda x: beam.Row(**x._asdict())),
equal_to([
beam.Row(animal='cat', weight=[1, 5]),
beam.Row(animal='dog', weight=[10]),
]))
def test_co_group_by(self):
with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions(
pickle_library='cloudpickle')) as p:
result = p | YamlTransform(
'''
type: composite
transforms:
# [START cogroupbykey_inputs]
- type: Create
name: CreateEmails
config:
elements:
- { name: "amy", email: "[email protected]" }
- { name: "carl", email: "[email protected]" }
- { name: "julia", email: "[email protected]" }
- { name: "carl", email: "[email protected]" }
- type: Create
name: CreatePhones
config:
elements:
- { name: "amy", phone: "111-222-3333" }
- { name: "james", phone: "222-333-4444" }
- { name: "amy", phone: "333-444-5555" }
- { name: "carl", phone: "444-555-6666" }
# [END cogroupbykey_inputs]
# [START cogroupbykey]
- type: MapToFields
name: PrepareEmails
input: CreateEmails
config:
language: python
fields:
name: name
email: "[email]"
phone: "[]"
- type: MapToFields
name: PreparePhones
input: CreatePhones
config:
language: python
fields:
name: name
email: "[]"
phone: "[phone]"
- type: Combine
name: CoGropuBy
input: [PrepareEmails, PreparePhones]
config:
group_by: [name]
combine:
email: concat
phone: concat
- type: MapToFields
name: FormatResults
input: CoGropuBy
config:
language: python
fields:
formatted:
"'%s; %s; %s' % (name, sorted(email), sorted(phone))"
# [END cogroupbykey]
output: FormatResults
''')
assert_that(
result | beam.Map(lambda x: x.formatted),
equal_to([
# [START cogroupbykey_formatted_outputs]
"amy; ['[email protected]']; ['111-222-3333', '333-444-5555']",
"carl; ['[email protected]', '[email protected]']; ['444-555-6666']",
"james; []; ['222-333-4444']",
"julia; ['[email protected]']; []",
# [END cogroupbykey_formatted_outputs]
]))
def test_combine_ref(self):
with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions(
pickle_library='cloudpickle')) as p:
elements = p | beam.Create(DATA)
result = elements | YamlTransform(
'''
# [START combine_simple_py]
type: Combine
config:
language: python
group_by: animal
combine:
biggest:
fn:
type: 'apache_beam.transforms.combiners.TopCombineFn'
config:
n: 2
value: weight
# [END combine_simple_py]
''')
assert_that(
result | beam.Map(lambda x: beam.Row(**x._asdict())),
equal_to([
beam.Row(animal='cat', biggest=[5, 1]),
beam.Row(animal='dog', biggest=[10]),
]))
def test_combine_globally(self):
with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions(
pickle_library='cloudpickle')) as p:
elements = p | beam.Create(DATA)
result = elements | YamlTransform(
'''
# [START combine_global_sum]
type: Combine
config:
group_by: []
combine:
weight: sum
# [END combine_global_sum]
''')
assert_that(
result | beam.Map(lambda x: beam.Row(**x._asdict())),
equal_to([
beam.Row(weight=16),
]))
def test_combine_per_key(self):
with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions(
pickle_library='cloudpickle')) as p:
elements = p | beam.Create(DATA)
result = elements | YamlTransform(
'''
# [START combine_per_key]
type: Combine
config:
group_by: [animal]
combine:
total_weight:
fn: sum
value: weight
average_weight:
fn: mean
value: weight
# [END combine_per_key]
''')
assert_that(
result | beam.Map(lambda x: beam.Row(**x._asdict())),
equal_to([
beam.Row(animal='cat', total_weight=6, average_weight=3),
beam.Row(animal='dog', total_weight=10, average_weight=10),
]))
def test_flatten(self):
with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions(
pickle_library='cloudpickle')) as p:
result = p | YamlTransform(
'''
type: composite
transforms:
- type: Create
name: SomeProducingTransform
config:
elements: [a, b, c]
- type: Create
name: AnotherProducingTransform
config:
elements: [d, e, f]
# [START model_multiple_pcollections_flatten]
- type: Flatten
input: [SomeProducingTransform, AnotherProducingTransform]
# [END model_multiple_pcollections_flatten]
output: Flatten
''')
assert_that(
result | beam.Map(lambda x: x.element),
equal_to(['a', 'b', 'c', 'd', 'e', 'f']))
def test_expode(self):
with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions(
pickle_library='cloudpickle')) as p:
result = p | YamlTransform(
'''
type: composite
transforms:
- type: Create
name: SomeProducingTransform
config:
elements:
- {line: "a aa"}
- {line: "b"}
# [START model_multiple_output_dofn]
- type: MapToFields
input: SomeProducingTransform
config:
language: python
fields:
word: "line.split()"
- type: Explode
input: MapToFields
config:
fields: word
# [END model_multiple_output_dofn]
output: Explode
''')
assert_that(
result | beam.Map(lambda x: x.word), equal_to(['a', 'aa', 'b']))
def test_schema_output_type(self):
with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions(
pickle_library='cloudpickle')) as p:
elements = p | beam.Create(DATA)
result = elements | YamlTransform(
'''
# [START schema_output_type]
type: MapToFields
config:
language: python
fields:
new_field:
expression: "hex(weight)"
output_type: { "type": "string" }
# [END schema_output_type]
''')
assert_that(
result | beam.Map(lambda x: x.new_field),
equal_to(['0x1', '0x5', '0xa']))
def test_fixed_windows(self):
with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions(
pickle_library='cloudpickle')) as p:
elements = p | beam.Create(DATA)
_ = elements | YamlTransform(
'''
# [START setting_fixed_windows]
type: WindowInto
windowing:
type: fixed
size: 60s
# [END setting_fixed_windows]
''')
def test_sliding_windows(self):
with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions(
pickle_library='cloudpickle')) as p:
elements = p | beam.Create(DATA)
_ = elements | YamlTransform(
'''
# [START setting_sliding_windows]
type: WindowInto
windowing:
type: sliding
size: 5m
period: 30s
# [END setting_sliding_windows]
''')
def test_session_windows(self):
with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions(
pickle_library='cloudpickle')) as p:
elements = p | beam.Create(DATA)
_ = elements | YamlTransform(
'''
# [START setting_session_windows]
type: WindowInto
windowing:
type: sessions
gap: 60s
# [END setting_session_windows]
''')
def test_global_windows(self):
with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions(
pickle_library='cloudpickle')) as p:
elements = p | beam.Create(DATA)
_ = elements | YamlTransform(
'''
# [START setting_global_window]
type: WindowInto
windowing:
type: global
# [END setting_global_window]
''')
def test_assign_timestamps(self):
with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions(
pickle_library='cloudpickle')) as p:
elements = p | beam.Create([
beam.Row(external_timestamp_field='2024-01-01', value=1),
beam.Row(external_timestamp_field='2024-01-01', value=2),
beam.Row(external_timestamp_field='2024-01-02', value=10),
])
_ = elements | YamlTransform(
'''
# [START setting_timestamp]
type: AssignTimestamps
config:
language: python
timestamp:
callable: |
import datetime
def extract_timestamp(x):
raw = datetime.datetime.strptime(
x.external_timestamp_field, "%Y-%m-%d")
return raw.astimezone(datetime.timezone.utc)
# [END setting_timestamp]
''')
def test_partition(self):
with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions(
pickle_library='cloudpickle')) as p:
elements = p | beam.Create([
beam.Row(percentile=1),
beam.Row(percentile=20),
beam.Row(percentile=90),
])
_ = elements | YamlTransform(
'''
# [START model_multiple_pcollections_partition]
type: Partition
config:
by: str(percentile // 10)
language: python
outputs: ["0", "1", "2", "3", "4", "5", "6", "7", "8", "9", "10"]
# [END model_multiple_pcollections_partition]
''')
if __name__ == '__main__':
logging.getLogger().setLevel(logging.INFO)
unittest.main()