-
Notifications
You must be signed in to change notification settings - Fork 4.3k
/
Copy pathexternal_test.py
813 lines (694 loc) · 31.1 KB
/
external_test.py
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
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
#
# 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.
#
"""Unit tests for the transform.external classes."""
# pytype: skip-file
import dataclasses
import logging
import os
import tempfile
import typing
import unittest
import mock
import apache_beam as beam
from apache_beam import Pipeline
from apache_beam.coders import RowCoder
from apache_beam.options.pipeline_options import PipelineOptions
from apache_beam.portability.api import beam_expansion_api_pb2
from apache_beam.portability.api import external_transforms_pb2
from apache_beam.portability.api import schema_pb2
from apache_beam.runners import pipeline_context
from apache_beam.runners.portability import expansion_service
from apache_beam.runners.portability.expansion_service_test import FibTransform
from apache_beam.testing.util import assert_that
from apache_beam.testing.util import equal_to
from apache_beam.transforms import external
from apache_beam.transforms.external import AnnotationBasedPayloadBuilder
from apache_beam.transforms.external import ImplicitSchemaPayloadBuilder
from apache_beam.transforms.external import JavaClassLookupPayloadBuilder
from apache_beam.transforms.external import JavaExternalTransform
from apache_beam.transforms.external import JavaJarExpansionService
from apache_beam.transforms.external import NamedTupleBasedPayloadBuilder
from apache_beam.transforms.external import SchemaTransformPayloadBuilder
from apache_beam.typehints import typehints
from apache_beam.typehints.native_type_compatibility import convert_to_beam_type
from apache_beam.utils import proto_utils
from apache_beam.utils.subprocess_server import JavaJarServer
from apache_beam.utils.subprocess_server import SubprocessServer
# Protect against environments where apitools library is not available.
# pylint: disable=wrong-import-order, wrong-import-position
try:
from apache_beam.runners.dataflow.internal import apiclient
except ImportError:
apiclient = None # type: ignore
# pylint: enable=wrong-import-order, wrong-import-position
def get_payload(cls):
payload = external_transforms_pb2.ExternalConfigurationPayload()
payload.ParseFromString(cls._payload)
return payload
class PayloadBase(object):
values = {
'integer_example': 1,
'boolean': True,
'string_example': 'thing',
'list_of_strings': ['foo', 'bar'],
'mapping': {
'key': 1.1
},
'optional_integer': None,
}
bytes_values = {
'integer_example': 1,
'boolean': True,
'string_example': 'thing',
'list_of_strings': ['foo', 'bar'],
'mapping': {
'key': 1.1
},
'optional_integer': None,
}
def get_payload_from_typing_hints(self, values):
"""Return ExternalConfigurationPayload based on python typing hints"""
raise NotImplementedError
def get_payload_from_beam_typehints(self, values):
"""Return ExternalConfigurationPayload based on beam typehints"""
raise NotImplementedError
def test_typing_payload_builder(self):
result = self.get_payload_from_typing_hints(self.values)
decoded = RowCoder(result.schema).decode(result.payload)
for key, value in self.values.items():
self.assertEqual(getattr(decoded, key), value)
def test_typehints_payload_builder(self):
result = self.get_payload_from_typing_hints(self.values)
decoded = RowCoder(result.schema).decode(result.payload)
for key, value in self.values.items():
self.assertEqual(getattr(decoded, key), value)
def test_optional_error(self):
"""
value can only be None if typehint is Optional
"""
with self.assertRaises(ValueError):
self.get_payload_from_typing_hints({k: None for k in self.values})
class ExternalTuplePayloadTest(PayloadBase, unittest.TestCase):
def get_payload_from_typing_hints(self, values):
TestSchema = typing.NamedTuple(
'TestSchema',
[
('integer_example', int),
('boolean', bool),
('string_example', str),
('list_of_strings', typing.List[str]),
('mapping', typing.Mapping[str, float]),
('optional_integer', typing.Optional[int]),
])
builder = NamedTupleBasedPayloadBuilder(TestSchema(**values))
return builder.build()
def get_payload_from_beam_typehints(self, values):
raise unittest.SkipTest(
"Beam typehints cannot be used with "
"typing.NamedTuple")
class ExternalImplicitPayloadTest(unittest.TestCase):
"""
ImplicitSchemaPayloadBuilder works very differently than the other payload
builders
"""
def test_implicit_payload_builder(self):
builder = ImplicitSchemaPayloadBuilder(PayloadBase.values)
result = builder.build()
decoded = RowCoder(result.schema).decode(result.payload)
for key, value in PayloadBase.values.items():
# Note the default value in the getattr call.
# ImplicitSchemaPayloadBuilder omits fields with valu=None since their
# type cannot be inferred.
self.assertEqual(getattr(decoded, key, None), value)
def test_implicit_payload_builder_with_bytes(self):
values = PayloadBase.bytes_values
builder = ImplicitSchemaPayloadBuilder(values)
result = builder.build()
decoded = RowCoder(result.schema).decode(result.payload)
for key, value in PayloadBase.values.items():
# Note the default value in the getattr call.
# ImplicitSchemaPayloadBuilder omits fields with valu=None since their
# type cannot be inferred.
self.assertEqual(getattr(decoded, key, None), value)
# Verify we have not modified a cached type (BEAM-10766)
# TODO(BEAM-7372): Remove when bytes coercion code is removed.
self.assertEqual(
typehints.List[bytes], convert_to_beam_type(typing.List[bytes]))
class ExternalTransformTest(unittest.TestCase):
def test_pipeline_generation(self):
pipeline = beam.Pipeline()
_ = (
pipeline
| beam.Create(['a', 'b'])
| beam.ExternalTransform(
'beam:transforms:xlang:test:prefix',
ImplicitSchemaPayloadBuilder({'data': '0'}),
expansion_service.ExpansionServiceServicer()))
proto, _ = pipeline.to_runner_api(return_context=True)
pipeline_from_proto = Pipeline.from_runner_api(
proto, pipeline.runner, pipeline._options)
# Original pipeline has the un-expanded external transform
self.assertEqual([], pipeline.transforms_stack[0].parts[1].parts)
# new pipeline has the expanded external transform
self.assertNotEqual([],
pipeline_from_proto.transforms_stack[0].parts[1].parts)
self.assertEqual(
'ExternalTransform(beam:transforms:xlang:test:prefix)/TestLabel',
pipeline_from_proto.transforms_stack[0].parts[1].parts[0].full_label)
@unittest.skipIf(apiclient is None, 'GCP dependencies are not installed')
def test_pipeline_generation_with_runner_overrides(self):
pipeline_properties = [
'--job_name=test-job',
'--project=test-project',
'--temp_location=gs://beam/tmp',
'--no_auth',
'--dry_run=True',
'--sdk_location=container',
'--runner=DataflowRunner',
'--streaming',
'--region=us-central1'
]
with beam.Pipeline(options=PipelineOptions(pipeline_properties)) as p:
_ = (
p
| beam.io.ReadFromPubSub(
subscription=
'projects/dummy-project/subscriptions/dummy-subscription')
| beam.ExternalTransform(
'beam:transforms:xlang:test:prefix',
ImplicitSchemaPayloadBuilder({'data': '0'}),
expansion_service.ExpansionServiceServicer()))
pipeline_proto, _ = p.to_runner_api(return_context=True)
pubsub_read_transform = None
external_transform = None
proto_transforms = pipeline_proto.components.transforms
for id in proto_transforms:
if 'beam:transforms:xlang:test:prefix' in proto_transforms[
id].unique_name:
external_transform = proto_transforms[id]
if 'ReadFromPubSub' in proto_transforms[id].unique_name:
pubsub_read_transform = proto_transforms[id]
if not (pubsub_read_transform and external_transform):
raise ValueError(
'Could not find an external transform and the PubSub read transform '
'in the pipeline')
self.assertEqual(1, len(list(pubsub_read_transform.outputs.values())))
self.assertEqual(
list(pubsub_read_transform.outputs.values()),
list(external_transform.inputs.values()))
def test_payload(self):
with beam.Pipeline() as p:
res = (
p
| beam.Create(['a', 'bb'], reshuffle=False)
| beam.ExternalTransform(
'payload', b's', expansion_service.ExpansionServiceServicer()))
assert_that(res, equal_to(['as', 'bbs']))
def test_output_coder(self):
external_transform = beam.ExternalTransform(
'map_to_union_types',
None,
expansion_service.ExpansionServiceServicer()).with_output_types(int)
with beam.Pipeline() as p:
res = (p | beam.Create([2, 2], reshuffle=False) | external_transform)
assert_that(res, equal_to([2, 2]))
context = pipeline_context.PipelineContext(
external_transform._expanded_components)
self.assertEqual(len(external_transform._expanded_transform.outputs), 1)
for _, pcol_id in external_transform._expanded_transform.outputs.items():
pcol = context.pcollections.get_by_id(pcol_id)
self.assertEqual(pcol.element_type, int)
def test_no_output_coder(self):
external_transform = beam.ExternalTransform(
'map_to_union_types',
None,
expansion_service.ExpansionServiceServicer())
with beam.Pipeline() as p:
res = (p | beam.Create([2, 2], reshuffle=False) | external_transform)
assert_that(res, equal_to([2, 2]))
context = pipeline_context.PipelineContext(
external_transform._expanded_components)
self.assertEqual(len(external_transform._expanded_transform.outputs), 1)
for _, pcol_id in external_transform._expanded_transform.outputs.items():
pcol = context.pcollections.get_by_id(pcol_id)
self.assertEqual(pcol.element_type, typehints.Any)
def test_nested(self):
with beam.Pipeline() as p:
assert_that(p | FibTransform(6), equal_to([8]))
def test_external_empty_spec_translation(self):
pipeline = beam.Pipeline()
external_transform = beam.ExternalTransform(
'beam:transforms:xlang:test:prefix',
ImplicitSchemaPayloadBuilder({'data': '0'}),
expansion_service.ExpansionServiceServicer())
_ = (pipeline | beam.Create(['a', 'b']) | external_transform)
pipeline.run().wait_until_finish()
external_transform_label = (
'ExternalTransform(beam:transforms:xlang:test:prefix)/TestLabel')
for transform in external_transform._expanded_components.transforms.values(
):
# We clear the spec of one of the external transforms.
if transform.unique_name == external_transform_label:
transform.spec.Clear()
context = pipeline_context.PipelineContext()
proto_pipeline = pipeline.to_runner_api(context=context)
proto_transform = None
for transform in proto_pipeline.components.transforms.values():
if (transform.unique_name ==
'ExternalTransform(beam:transforms:xlang:test:prefix)/TestLabel'):
proto_transform = transform
self.assertIsNotNone(proto_transform)
self.assertTrue(str(proto_transform).strip().find('spec {') == -1)
def test_unique_name(self):
p = beam.Pipeline()
_ = p | FibTransform(6)
proto = p.to_runner_api()
xforms = [x.unique_name for x in proto.components.transforms.values()]
self.assertEqual(
len(set(xforms)), len(xforms), msg='Transform names are not unique.')
pcolls = [x.unique_name for x in proto.components.pcollections.values()]
self.assertEqual(
len(set(pcolls)), len(pcolls), msg='PCollection names are not unique.')
def test_external_transform_finder_non_leaf(self):
pipeline = beam.Pipeline()
_ = (
pipeline
| beam.Create(['a', 'b'])
| beam.ExternalTransform(
'beam:transforms:xlang:test:prefix',
ImplicitSchemaPayloadBuilder({'data': '0'}),
expansion_service.ExpansionServiceServicer())
| beam.Map(lambda x: x))
pipeline.run().wait_until_finish()
self.assertTrue(pipeline.contains_external_transforms)
def test_external_transform_finder_leaf(self):
pipeline = beam.Pipeline()
_ = (
pipeline
| beam.Create(['a', 'b'])
| beam.ExternalTransform(
'beam:transforms:xlang:test:nooutput',
ImplicitSchemaPayloadBuilder({'data': '0'}),
expansion_service.ExpansionServiceServicer()))
pipeline.run().wait_until_finish()
self.assertTrue(pipeline.contains_external_transforms)
def test_sanitize_java_traceback(self):
error_string = '''
java.lang.RuntimeException: ACTUAL \n MULTILINE \n ERROR
\tat org.apache.beam.sdk.expansion.service.ExpansionService$TransformProviderForBuilder.getTransform(ExpansionService.java:308)
\tat org.apache.beam.sdk.expansion.service.TransformProvider.apply(TransformProvider.java:121)
\tat org.apache.beam.sdk.expansion.service.ExpansionService.expand(ExpansionService.java:627)
\tat org.apache.beam.sdk.expansion.service.ExpansionService.expand(ExpansionService.java:729)
\tat org.apache.beam.model.expansion.v1.ExpansionServiceGrpc$MethodHandlers.invoke(ExpansionServiceGrpc.java:306)
\tat org.apache.beam.vendor.grpc.v1p69p0.io.grpc.stub.ServerCalls$UnaryServerCallHandler$UnaryServerCallListener.onHalfClose(ServerCalls.java:182)
\tat org.apache.beam.vendor.grpc.v1p69p0.io.grpc.internal.ServerCallImpl$ServerStreamListenerImpl.halfClosed(ServerCallImpl.java:351)
\tat org.apache.beam.vendor.grpc.v1p69p0.io.grpc.internal.ServerImpl$JumpToApplicationThreadServerStreamListener$1HalfClosed.runInContext(ServerImpl.java:861)
\tat org.apache.beam.vendor.grpc.v1p69p0.io.grpc.internal.ContextRunnable.run(ContextRunnable.java:37)
\tat org.apache.beam.vendor.grpc.v1p69p0.io.grpc.internal.SerializingExecutor.run(SerializingExecutor.java:133)
\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
\tat java.lang.Thread.run(Thread.java:748)
Caused by: java.lang.IllegalArgumentException: Received unknown SQL Dialect 'X'. Known dialects: [zetasql, calcite]
\tat org.apache.beam.sdk.extensions.sql.expansion.ExternalSqlTransformRegistrar$Builder.buildExternal(ExternalSqlTransformRegistrar.java:73)
\tat org.apache.beam.sdk.extensions.sql.expansion.ExternalSqlTransformRegistrar$Builder.buildExternal(ExternalSqlTransformRegistrar.java:63)
\tat org.apache.beam.sdk.expansion.service.ExpansionService$TransformProviderForBuilder.getTransform(ExpansionService.java:303)
\t... 12 more
'''.strip()
core_msg = 'java.lang.RuntimeException: ACTUAL \n MULTILINE \n ERROR'
self.assertEqual(
f"{error_string}\n\n{core_msg}",
external._sanitize_java_traceback(error_string))
class ExternalAnnotationPayloadTest(PayloadBase, unittest.TestCase):
def get_payload_from_typing_hints(self, values):
class AnnotatedTransform(beam.ExternalTransform):
URN = 'beam:external:fakeurn:v1'
def __init__(
self,
integer_example: int,
boolean: bool,
string_example: str,
list_of_strings: typing.List[str],
mapping: typing.Mapping[str, float],
optional_integer: typing.Optional[int] = None,
expansion_service=None):
super().__init__(
self.URN,
AnnotationBasedPayloadBuilder(
self,
integer_example=integer_example,
boolean=boolean,
string_example=string_example,
list_of_strings=list_of_strings,
mapping=mapping,
optional_integer=optional_integer,
),
expansion_service)
return get_payload(AnnotatedTransform(**values))
def get_payload_from_beam_typehints(self, values):
class AnnotatedTransform(beam.ExternalTransform):
URN = 'beam:external:fakeurn:v1'
def __init__(
self,
integer_example: int,
boolean: bool,
string_example: str,
list_of_strings: typehints.List[str],
mapping: typehints.Dict[str, float],
optional_integer: typehints.Optional[int] = None,
expansion_service=None):
super().__init__(
self.URN,
AnnotationBasedPayloadBuilder(
self,
integer_example=integer_example,
boolean=boolean,
string_example=string_example,
list_of_strings=list_of_strings,
mapping=mapping,
optional_integer=optional_integer,
),
expansion_service)
return get_payload(AnnotatedTransform(**values))
class ExternalDataclassesPayloadTest(PayloadBase, unittest.TestCase):
def get_payload_from_typing_hints(self, values):
@dataclasses.dataclass
class DataclassTransform(beam.ExternalTransform):
URN = 'beam:external:fakeurn:v1'
integer_example: int
boolean: bool
string_example: str
list_of_strings: typing.List[str]
mapping: typing.Mapping[str, float] = dataclasses.field(default=dict)
optional_integer: typing.Optional[int] = None
expansion_service: dataclasses.InitVar[typing.Optional[str]] = None
return get_payload(DataclassTransform(**values))
def get_payload_from_beam_typehints(self, values):
@dataclasses.dataclass
class DataclassTransform(beam.ExternalTransform):
URN = 'beam:external:fakeurn:v1'
integer_example: int
boolean: bool
string_example: str
list_of_strings: typehints.List[str]
mapping: typehints.Dict[str, float] = {}
optional_integer: typehints.Optional[int] = None
expansion_service: dataclasses.InitVar[typehints.Optional[str]] = None
return get_payload(DataclassTransform(**values))
class SchemaTransformPayloadBuilderTest(unittest.TestCase):
def test_build_payload(self):
ComplexType = typing.NamedTuple(
"ComplexType", [
("str_sub_field", str),
("int_sub_field", int),
])
payload_builder = SchemaTransformPayloadBuilder(
identifier='dummy_id',
str_field='aaa',
int_field=123,
object_field=ComplexType(str_sub_field="bbb", int_sub_field=456))
payload_bytes = payload_builder.payload()
payload_from_bytes = proto_utils.parse_Bytes(
payload_bytes, external_transforms_pb2.SchemaTransformPayload)
self.assertEqual('dummy_id', payload_from_bytes.identifier)
expected_coder = RowCoder(payload_from_bytes.configuration_schema)
schema_transform_config = expected_coder.decode(
payload_from_bytes.configuration_row)
self.assertEqual('aaa', schema_transform_config.str_field)
self.assertEqual(123, schema_transform_config.int_field)
self.assertEqual('bbb', schema_transform_config.object_field.str_sub_field)
self.assertEqual(456, schema_transform_config.object_field.int_sub_field)
class SchemaAwareExternalTransformTest(unittest.TestCase):
class MockDiscoveryService:
# define context manager enter and exit functions
def __enter__(self):
return self
def __exit__(self, unusued1, unused2, unused3):
pass
def DiscoverSchemaTransform(self, unused_request=None):
test_config = beam_expansion_api_pb2.SchemaTransformConfig(
config_schema=schema_pb2.Schema(
fields=[
schema_pb2.Field(
name="str_field",
type=schema_pb2.FieldType(atomic_type="STRING")),
schema_pb2.Field(
name="int_field",
type=schema_pb2.FieldType(atomic_type="INT64"))
],
id="test-id"),
input_pcollection_names=["input"],
output_pcollection_names=["output"])
return beam_expansion_api_pb2.DiscoverSchemaTransformResponse(
schema_transform_configs={"test_schematransform": test_config})
@mock.patch("apache_beam.transforms.external.ExternalTransform.service")
def test_discover_one_config(self, mock_service):
_mock = self.MockDiscoveryService()
mock_service.return_value = _mock
config = beam.SchemaAwareExternalTransform.discover_config(
"test_service", name="test_schematransform")
self.assertEqual(config.outputs[0], "output")
self.assertEqual(config.inputs[0], "input")
self.assertEqual(config.identifier, "test_schematransform")
@mock.patch("apache_beam.transforms.external.ExternalTransform.service")
def test_discover_one_config_fails_with_no_configs_found(self, mock_service):
mock_service.return_value = self.MockDiscoveryService()
with self.assertRaises(ValueError):
beam.SchemaAwareExternalTransform.discover_config(
"test_service", name="non_existent")
@mock.patch("apache_beam.transforms.external.ExternalTransform.service")
def test_rearrange_kwargs_based_on_discovery(self, mock_service):
mock_service.return_value = self.MockDiscoveryService()
identifier = "test_schematransform"
expansion_service = "test_service"
kwargs = {"int_field": 0, "str_field": "str"}
transform = beam.SchemaAwareExternalTransform(
identifier=identifier,
expansion_service=expansion_service,
rearrange_based_on_discovery=True,
**kwargs)
payload = transform._payload_builder.build()
ordered_fields = [f.name for f in payload.configuration_schema.fields]
schematransform_config = beam.SchemaAwareExternalTransform.discover_config(
expansion_service, identifier)
external_config_fields = schematransform_config.configuration_schema._fields
self.assertNotEqual(tuple(kwargs.keys()), external_config_fields)
self.assertEqual(tuple(ordered_fields), external_config_fields)
class JavaClassLookupPayloadBuilderTest(unittest.TestCase):
def _verify_row(self, schema, row_payload, expected_values):
row = RowCoder(schema).decode(row_payload)
for attr_name, expected_value in expected_values.items():
self.assertTrue(hasattr(row, attr_name))
value = getattr(row, attr_name)
self.assertEqual(expected_value, value)
def test_build_payload_with_constructor(self):
payload_builder = JavaClassLookupPayloadBuilder('dummy_class_name')
payload_builder.with_constructor('abc', 123, str_field='def', int_field=456)
payload_bytes = payload_builder.payload()
payload_from_bytes = proto_utils.parse_Bytes(
payload_bytes, external_transforms_pb2.JavaClassLookupPayload)
self.assertTrue(
isinstance(
payload_from_bytes, external_transforms_pb2.JavaClassLookupPayload))
self.assertFalse(payload_from_bytes.constructor_method)
self._verify_row(
payload_from_bytes.constructor_schema,
payload_from_bytes.constructor_payload, {
'ignore0': 'abc',
'ignore1': 123,
'str_field': 'def',
'int_field': 456
})
def test_build_payload_with_constructor_method(self):
payload_builder = JavaClassLookupPayloadBuilder('dummy_class_name')
payload_builder.with_constructor_method(
'dummy_constructor_method', 'abc', 123, str_field='def', int_field=456)
payload_bytes = payload_builder.payload()
payload_from_bytes = proto_utils.parse_Bytes(
payload_bytes, external_transforms_pb2.JavaClassLookupPayload)
self.assertTrue(
isinstance(
payload_from_bytes, external_transforms_pb2.JavaClassLookupPayload))
self.assertEqual(
'dummy_constructor_method', payload_from_bytes.constructor_method)
self._verify_row(
payload_from_bytes.constructor_schema,
payload_from_bytes.constructor_payload, {
'ignore0': 'abc',
'ignore1': 123,
'str_field': 'def',
'int_field': 456
})
def test_build_payload_with_builder_methods(self):
payload_builder = JavaClassLookupPayloadBuilder('dummy_class_name')
payload_builder.with_constructor('abc', 123, str_field='def', int_field=456)
payload_builder.add_builder_method(
'builder_method1', 'abc1', 1234, str_field1='abc2', int_field1=2345)
payload_builder.add_builder_method(
'builder_method2', 'abc3', 3456, str_field2='abc4', int_field2=4567)
payload_bytes = payload_builder.payload()
payload_from_bytes = proto_utils.parse_Bytes(
payload_bytes, external_transforms_pb2.JavaClassLookupPayload)
self.assertTrue(
isinstance(
payload_from_bytes, external_transforms_pb2.JavaClassLookupPayload))
self._verify_row(
payload_from_bytes.constructor_schema,
payload_from_bytes.constructor_payload, {
'ignore0': 'abc',
'ignore1': 123,
'str_field': 'def',
'int_field': 456
})
self.assertEqual(2, len(payload_from_bytes.builder_methods))
builder_method = payload_from_bytes.builder_methods[0]
self.assertTrue(
isinstance(builder_method, external_transforms_pb2.BuilderMethod))
self.assertEqual('builder_method1', builder_method.name)
self._verify_row(
builder_method.schema,
builder_method.payload,
{
'ignore0': 'abc1',
'ignore1': 1234,
'str_field1': 'abc2',
'int_field1': 2345
})
builder_method = payload_from_bytes.builder_methods[1]
self.assertTrue(
isinstance(builder_method, external_transforms_pb2.BuilderMethod))
self.assertEqual('builder_method2', builder_method.name)
self._verify_row(
builder_method.schema,
builder_method.payload,
{
'ignore0': 'abc3',
'ignore1': 3456,
'str_field2': 'abc4',
'int_field2': 4567
})
def test_build_payload_with_constructor_twice_fails(self):
payload_builder = JavaClassLookupPayloadBuilder('dummy_class_name')
payload_builder.with_constructor('abc')
with self.assertRaises(ValueError):
payload_builder.with_constructor('def')
def test_implicit_builder_with_constructor(self):
constructor_transform = (
JavaExternalTransform('org.pkg.MyTransform')('abc').withIntProperty(5))
payload_bytes = constructor_transform._payload_builder.payload()
payload_from_bytes = proto_utils.parse_Bytes(
payload_bytes, external_transforms_pb2.JavaClassLookupPayload)
self.assertEqual('org.pkg.MyTransform', payload_from_bytes.class_name)
self._verify_row(
payload_from_bytes.constructor_schema,
payload_from_bytes.constructor_payload, {'ignore0': 'abc'})
builder_method = payload_from_bytes.builder_methods[0]
self.assertEqual('withIntProperty', builder_method.name)
self._verify_row(
builder_method.schema, builder_method.payload, {'ignore0': 5})
def test_implicit_builder_with_constructor_method(self):
constructor_transform = JavaExternalTransform('org.pkg.MyTransform').of(
str_field='abc').withProperty(int_field=1234).build()
payload_bytes = constructor_transform._payload_builder.payload()
payload_from_bytes = proto_utils.parse_Bytes(
payload_bytes, external_transforms_pb2.JavaClassLookupPayload)
self.assertEqual('of', payload_from_bytes.constructor_method)
self._verify_row(
payload_from_bytes.constructor_schema,
payload_from_bytes.constructor_payload, {'str_field': 'abc'})
with_property_method = payload_from_bytes.builder_methods[0]
self.assertEqual('withProperty', with_property_method.name)
self._verify_row(
with_property_method.schema,
with_property_method.payload, {'int_field': 1234})
build_method = payload_from_bytes.builder_methods[1]
self.assertEqual('build', build_method.name)
self._verify_row(build_method.schema, build_method.payload, {})
class JavaJarExpansionServiceTest(unittest.TestCase):
def setUp(self):
SubprocessServer._cache._live_owners = set()
def test_classpath(self):
with tempfile.TemporaryDirectory() as temp_dir:
try:
# Avoid having to prefix everything in our test strings.
oldwd = os.getcwd()
os.chdir(temp_dir)
# Touch some files for globing.
with open('a1.jar', 'w') as _:
pass
service = JavaJarExpansionService(
'main.jar', classpath=['a*.jar', 'b.jar'])
self.assertEqual(
service._default_args(),
['{{PORT}}', '--filesToStage=main.jar,a1.jar,b.jar'])
finally:
os.chdir(oldwd)
@mock.patch.object(JavaJarServer, 'local_jar')
def test_classpath_with_url(self, local_jar):
def _side_effect_fn(path):
return path[path.rindex('/') + 1:]
local_jar.side_effect = _side_effect_fn
with tempfile.TemporaryDirectory() as temp_dir:
try:
# Avoid having to prefix everything in our test strings.
oldwd = os.getcwd()
os.chdir(temp_dir)
service = JavaJarExpansionService(
'main.jar', classpath=['https://dummy_path/dummyjar.jar'])
self.assertEqual(
service._default_args(),
['{{PORT}}', '--filesToStage=main.jar,dummyjar.jar'])
finally:
os.chdir(oldwd)
@mock.patch.object(JavaJarServer, 'local_jar')
def test_classpath_with_gradle_artifact(self, local_jar):
def _side_effect_fn(path):
return path[path.rindex('/') + 1:]
local_jar.side_effect = _side_effect_fn
with tempfile.TemporaryDirectory() as temp_dir:
try:
# Avoid having to prefix everything in our test strings.
oldwd = os.getcwd()
os.chdir(temp_dir)
service = JavaJarExpansionService(
'main.jar', classpath=['dummy_group:dummy_artifact:dummy_version'])
self.assertEqual(
service._default_args(),
[
'{{PORT}}',
'--filesToStage=main.jar,dummy_artifact-dummy_version.jar'
])
finally:
os.chdir(oldwd)
def test_classpath_with_glob(self):
with tempfile.TemporaryDirectory() as temp_dir:
try:
# Avoid having to prefix everything in our test strings.
oldwd = os.getcwd()
os.chdir(temp_dir)
# Touch some files for globing.
with open('a1.jar', 'w') as _:
pass
service = JavaJarExpansionService(
'main.jar', classpath=['a*.jar', 'b.jar'])
self.assertEqual(
service._default_args(),
['{{PORT}}', '--filesToStage=main.jar,a1.jar,b.jar'])
finally:
os.chdir(oldwd)
if __name__ == '__main__':
logging.getLogger().setLevel(logging.INFO)
unittest.main()