github.com/apache/beam/sdks/v2@v2.48.2/python/apache_beam/transforms/validate_runner_xlang_test.py (about) 1 # 2 # Licensed to the Apache Software Foundation (ASF) under one or more 3 # contributor license agreements. See the NOTICE file distributed with 4 # this work for additional information regarding copyright ownership. 5 # The ASF licenses this file to You under the Apache License, Version 2.0 6 # (the "License"); you may not use this file except in compliance with 7 # the License. You may obtain a copy of the License at 8 # 9 # http://www.apache.org/licenses/LICENSE-2.0 10 # 11 # Unless required by applicable law or agreed to in writing, software 12 # distributed under the License is distributed on an "AS IS" BASIS, 13 # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 # See the License for the specific language governing permissions and 15 # limitations under the License. 16 # 17 18 """ 19 ########################################################### 20 Runner Validation Test Suite for Cross-language Transforms 21 ########################################################### 22 As per Beams's Portability Framework design, Cross-language transforms 23 should work out of the box. In spite of this, there always exists a 24 possibility of rough edges existing. It could be caused due to unpolished 25 implementation of any part of the execution code path, for example: 26 - Transform expansion [SDK] 27 - Pipeline construction [SDK] 28 - Cross-language artifact staging [Runner] 29 - Language specific serialization/deserialization of PCollection (and 30 other data types) [Runner/SDK] 31 32 In an effort to improve developer visibility into potential problems, 33 this test suite validates correct execution of 5 Core Beam transforms when 34 used as cross-language transforms within the Python SDK from any foreign SDK: 35 - ParDo 36 (https://beam.apache.org/documentation/programming-guide/#pardo) 37 - GroupByKey 38 (https://beam.apache.org/documentation/programming-guide/#groupbykey) 39 - CoGroupByKey 40 (https://beam.apache.org/documentation/programming-guide/#cogroupbykey) 41 - Combine 42 (https://beam.apache.org/documentation/programming-guide/#combine) 43 - Flatten 44 (https://beam.apache.org/documentation/programming-guide/#flatten) 45 - Partition 46 (https://beam.apache.org/documentation/programming-guide/#partition) 47 48 See Runner Validation Test Plan for Cross-language transforms at 49 https://docs.google.com/document/d/1xQp0ElIV84b8OCVz8CD2hvbiWdR8w4BvWxPTZJZA6NA 50 for further details. 51 """ 52 53 import logging 54 import os 55 import typing 56 import unittest 57 58 import pytest 59 60 import apache_beam as beam 61 from apache_beam.testing.test_pipeline import TestPipeline 62 from apache_beam.testing.util import assert_that 63 from apache_beam.testing.util import equal_to 64 from apache_beam.transforms.external import ImplicitSchemaPayloadBuilder 65 66 TEST_PREFIX_URN = "beam:transforms:xlang:test:prefix" 67 TEST_MULTI_URN = "beam:transforms:xlang:test:multi" 68 TEST_GBK_URN = "beam:transforms:xlang:test:gbk" 69 TEST_CGBK_URN = "beam:transforms:xlang:test:cgbk" 70 TEST_COMGL_URN = "beam:transforms:xlang:test:comgl" 71 TEST_COMPK_URN = "beam:transforms:xlang:test:compk" 72 TEST_FLATTEN_URN = "beam:transforms:xlang:test:flatten" 73 TEST_PARTITION_URN = "beam:transforms:xlang:test:partition" 74 75 76 class CrossLanguageTestPipelines(object): 77 def __init__(self, expansion_service=None): 78 self.expansion_service = expansion_service or ( 79 'localhost:%s' % os.environ.get('EXPANSION_PORT')) 80 81 def run_prefix(self, pipeline): 82 """ 83 Target transform - ParDo 84 (https://beam.apache.org/documentation/programming-guide/#pardo) 85 Test scenario - Mapping elements from a single input collection to a 86 single output collection 87 Boundary conditions checked - 88 - PCollection<?> to external transforms 89 - PCollection<?> from external transforms 90 """ 91 with pipeline as p: 92 res = ( 93 p 94 | beam.Create(['a', 'b']).with_output_types(str) 95 | beam.ExternalTransform( 96 TEST_PREFIX_URN, 97 ImplicitSchemaPayloadBuilder({'data': u'0'}), 98 self.expansion_service)) 99 assert_that(res, equal_to(['0a', '0b'])) 100 101 def run_multi_input_output_with_sideinput(self, pipeline): 102 """ 103 Target transform - ParDo 104 (https://beam.apache.org/documentation/programming-guide/#pardo) 105 Test scenario - Mapping elements from multiple input collections (main 106 and side) to multiple output collections (main and side) 107 Boundary conditions checked - 108 - PCollectionTuple to external transforms 109 - PCollectionTuple from external transforms 110 """ 111 with pipeline as p: 112 main1 = p | 'Main1' >> beam.Create( 113 ['a', 'bb'], reshuffle=False).with_output_types(str) 114 main2 = p | 'Main2' >> beam.Create( 115 ['x', 'yy', 'zzz'], reshuffle=False).with_output_types(str) 116 side = p | 'Side' >> beam.Create(['s']).with_output_types(str) 117 res = dict( 118 main1=main1, main2=main2, side=side) | beam.ExternalTransform( 119 TEST_MULTI_URN, None, self.expansion_service) 120 assert_that(res['main'], equal_to(['as', 'bbs', 'xs', 'yys', 'zzzs'])) 121 assert_that(res['side'], equal_to(['ss']), label='CheckSide') 122 123 def run_group_by_key(self, pipeline): 124 """ 125 Target transform - GroupByKey 126 (https://beam.apache.org/documentation/programming-guide/#groupbykey) 127 Test scenario - Grouping a collection of KV<K,V> to a collection of 128 KV<K, Iterable<V>> by key 129 Boundary conditions checked - 130 - PCollection<KV<?, ?>> to external transforms 131 - PCollection<KV<?, Iterable<?>>> from external transforms 132 """ 133 with pipeline as p: 134 res = ( 135 p 136 | beam.Create([(0, "1"), (0, "2"), 137 (1, "3")], reshuffle=False).with_output_types( 138 typing.Tuple[int, str]) 139 | beam.ExternalTransform(TEST_GBK_URN, None, self.expansion_service) 140 | beam.Map(lambda x: "{}:{}".format(x[0], ','.join(sorted(x[1]))))) 141 assert_that(res, equal_to(['0:1,2', '1:3'])) 142 143 def run_cogroup_by_key(self, pipeline): 144 """ 145 Target transform - CoGroupByKey 146 (https://beam.apache.org/documentation/programming-guide/#cogroupbykey) 147 Test scenario - Grouping multiple input collections with keys to a 148 collection of KV<K, CoGbkResult> by key 149 Boundary conditions checked - 150 - KeyedPCollectionTuple<?> to external transforms 151 - PCollection<KV<?, Iterable<?>>> from external transforms 152 """ 153 with pipeline as p: 154 col1 = p | 'create_col1' >> beam.Create( 155 [(0, "1"), (0, "2"), (1, "3")], reshuffle=False).with_output_types( 156 typing.Tuple[int, str]) 157 col2 = p | 'create_col2' >> beam.Create( 158 [(0, "4"), (1, "5"), (1, "6")], reshuffle=False).with_output_types( 159 typing.Tuple[int, str]) 160 res = ( 161 dict(col1=col1, col2=col2) 162 | beam.ExternalTransform(TEST_CGBK_URN, None, self.expansion_service) 163 | beam.Map(lambda x: "{}:{}".format(x[0], ','.join(sorted(x[1]))))) 164 assert_that(res, equal_to(['0:1,2,4', '1:3,5,6'])) 165 166 def run_combine_globally(self, pipeline): 167 """ 168 Target transform - Combine 169 (https://beam.apache.org/documentation/programming-guide/#combine) 170 Test scenario - Combining elements globally with a predefined simple 171 CombineFn 172 Boundary conditions checked - 173 - PCollection<?> to external transforms 174 - PCollection<?> from external transforms 175 """ 176 with pipeline as p: 177 res = ( 178 p 179 | beam.Create([1, 2, 3]).with_output_types(int) 180 | beam.ExternalTransform( 181 TEST_COMGL_URN, None, self.expansion_service)) 182 assert_that(res, equal_to([6])) 183 184 def run_combine_per_key(self, pipeline): 185 """ 186 Target transform - Combine 187 (https://beam.apache.org/documentation/programming-guide/#combine) 188 Test scenario - Combining elements per key with a predefined simple 189 merging function 190 Boundary conditions checked - 191 - PCollection<?> to external transforms 192 - PCollection<?> from external transforms 193 """ 194 with pipeline as p: 195 res = ( 196 p 197 | beam.Create([('a', 1), ('a', 2), 198 ('b', 3)]).with_output_types(typing.Tuple[str, int]) 199 | beam.ExternalTransform( 200 TEST_COMPK_URN, None, self.expansion_service)) 201 assert_that(res, equal_to([('a', 3), ('b', 3)])) 202 203 def run_flatten(self, pipeline): 204 """ 205 Target transform - Flatten 206 (https://beam.apache.org/documentation/programming-guide/#flatten) 207 Test scenario - Merging multiple collections into a single collection 208 Boundary conditions checked - 209 - PCollectionList<?> to external transforms 210 - PCollection<?> from external transforms 211 """ 212 with pipeline as p: 213 col1 = p | 'col1' >> beam.Create([1, 2, 3]).with_output_types(int) 214 col2 = p | 'col2' >> beam.Create([4, 5, 6]).with_output_types(int) 215 res = ((col1, col2) 216 | beam.ExternalTransform( 217 TEST_FLATTEN_URN, None, self.expansion_service)) 218 assert_that(res, equal_to([1, 2, 3, 4, 5, 6])) 219 220 def run_partition(self, pipeline): 221 """ 222 Target transform - Partition 223 (https://beam.apache.org/documentation/programming-guide/#partition) 224 Test scenario - Splitting a single collection into multiple collections 225 with a predefined simple PartitionFn 226 Boundary conditions checked - 227 - PCollection<?> to external transforms 228 - PCollectionList<?> from external transforms 229 """ 230 with pipeline as p: 231 res = ( 232 p 233 | beam.Create([1, 2, 3, 4, 5, 6]).with_output_types(int) 234 | beam.ExternalTransform( 235 TEST_PARTITION_URN, None, self.expansion_service)) 236 assert_that(res['0'], equal_to([2, 4, 6]), label='check_even') 237 assert_that(res['1'], equal_to([1, 3, 5]), label='check_odd') 238 239 240 @unittest.skipUnless( 241 os.environ.get('EXPANSION_PORT'), 242 "EXPANSION_PORT environment var is not provided.") 243 class ValidateRunnerXlangTest(unittest.TestCase): 244 _multiprocess_can_split_ = True 245 246 def create_pipeline(self): 247 test_pipeline = TestPipeline() 248 test_pipeline.not_use_test_runner_api = True 249 return test_pipeline 250 251 @pytest.mark.uses_java_expansion_service 252 @pytest.mark.uses_python_expansion_service 253 def test_prefix(self, test_pipeline=None): 254 CrossLanguageTestPipelines().run_prefix( 255 test_pipeline or self.create_pipeline()) 256 257 @pytest.mark.uses_java_expansion_service 258 @pytest.mark.uses_python_expansion_service 259 def test_multi_input_output_with_sideinput(self, test_pipeline=None): 260 CrossLanguageTestPipelines().run_multi_input_output_with_sideinput( 261 test_pipeline or self.create_pipeline()) 262 263 @pytest.mark.uses_java_expansion_service 264 @pytest.mark.uses_python_expansion_service 265 def test_group_by_key(self, test_pipeline=None): 266 CrossLanguageTestPipelines().run_group_by_key( 267 test_pipeline or self.create_pipeline()) 268 269 @pytest.mark.uses_java_expansion_service 270 @pytest.mark.uses_python_expansion_service 271 def test_cogroup_by_key(self, test_pipeline=None): 272 CrossLanguageTestPipelines().run_cogroup_by_key( 273 test_pipeline or self.create_pipeline()) 274 275 @pytest.mark.uses_java_expansion_service 276 @pytest.mark.uses_python_expansion_service 277 def test_combine_globally(self, test_pipeline=None): 278 CrossLanguageTestPipelines().run_combine_globally( 279 test_pipeline or self.create_pipeline()) 280 281 @pytest.mark.uses_java_expansion_service 282 @pytest.mark.uses_python_expansion_service 283 def test_combine_per_key(self, test_pipeline=None): 284 CrossLanguageTestPipelines().run_combine_per_key( 285 test_pipeline or self.create_pipeline()) 286 287 # TODO: enable after fixing BEAM-10507 288 # @pytest.mark.uses_java_expansion_service 289 # @pytest.mark.uses_python_expansion_service 290 def test_flatten(self, test_pipeline=None): 291 CrossLanguageTestPipelines().run_flatten( 292 test_pipeline or self.create_pipeline()) 293 294 @pytest.mark.uses_java_expansion_service 295 @pytest.mark.uses_python_expansion_service 296 def test_partition(self, test_pipeline=None): 297 CrossLanguageTestPipelines().run_partition( 298 test_pipeline or self.create_pipeline()) 299 300 301 if __name__ == '__main__': 302 logging.getLogger().setLevel(logging.INFO) 303 unittest.main()