5050from apache_beam .coders import coders
5151from apache_beam .metrics import MetricsFilter
5252from apache_beam .options .pipeline_options import PipelineOptions
53+ from apache_beam .options .pipeline_options import SetupOptions
5354from apache_beam .options .pipeline_options import StandardOptions
5455from apache_beam .options .pipeline_options import TypeOptions
5556from apache_beam .portability import common_urns
@@ -252,7 +253,7 @@ def test_co_group_by_key_on_unpickled(self):
252253
253254
254255class FakeSecret (beam .Secret ):
255- def __init__ (self , should_throw = False ):
256+ def __init__ (self , version_name = None , should_throw = False ):
256257 self ._secret = b'aKwI2PmqYFt2p5tNKCyBS5qYmHhHsGZcyZrnZQiQ-uE='
257258 self ._should_throw = should_throw
258259
@@ -273,6 +274,12 @@ def __init__(self, hmac_key_secret, key_coder, value_coder):
273274 super ().__init__ (hmac_key_secret , key_coder , value_coder )
274275
275276 def process (self , element ):
277+ final_elements = list (super ().process (element ))
278+ # Check if we're looking at the actual elements being encoded/decoded
279+ # There is also a gbk on assertEqual, which uses None as the key type.
280+ final_element_keys = [e for e in final_elements if e [0 ] in ['a' , 'b' , 'c' ]]
281+ if len (final_element_keys ) == 0 :
282+ return final_elements
276283 hmac_key , actual_elements = element
277284 if hmac_key not in self .known_hmacs :
278285 raise ValueError (f'GBK produced unencrypted value { hmac_key } ' )
@@ -286,7 +293,38 @@ def process(self, element):
286293 except InvalidToken :
287294 raise ValueError (f'GBK produced unencrypted value { e [1 ]} ' )
288295
289- return super ().process (element )
296+ return final_elements
297+
298+
299+ class SecretTest (unittest .TestCase ):
300+ @parameterized .expand ([
301+ param (
302+ secret_string = 'type:GcpSecret;version_name:my_secret/versions/latest' ,
303+ secret = GcpSecret ('my_secret/versions/latest' )),
304+ param (
305+ secret_string = 'type:GcpSecret;version_name:foo' ,
306+ secret = GcpSecret ('foo' )),
307+ param (
308+ secret_string = 'type:gcpsecreT;version_name:my_secret/versions/latest' ,
309+ secret = GcpSecret ('my_secret/versions/latest' )),
310+ ])
311+ def test_secret_manager_parses_correctly (self , secret_string , secret ):
312+ self .assertEqual (secret , Secret .parse_secret_option (secret_string ))
313+
314+ @parameterized .expand ([
315+ param (
316+ secret_string = 'version_name:foo' ,
317+ exception_str = 'must contain a valid type parameter' ),
318+ param (
319+ secret_string = 'type:gcpsecreT' ,
320+ exception_str = 'missing 1 required positional argument' ),
321+ param (
322+ secret_string = 'type:gcpsecreT;version_name:foo;extra:val' ,
323+ exception_str = 'Invalid secret parameter extra' ),
324+ ])
325+ def test_secret_manager_throws_on_invalid (self , secret_string , exception_str ):
326+ with self .assertRaisesRegex (Exception , exception_str ):
327+ Secret .parse_secret_option (secret_string )
290328
291329
292330class GroupByEncryptedKeyTest (unittest .TestCase ):
@@ -318,7 +356,9 @@ def setUp(self):
318356 'data' : Secret .generate_secret_bytes ()
319357 }
320358 })
321- self .gcp_secret = GcpSecret (f'{ self .secret_path } /versions/latest' )
359+ version_name = f'{ self .secret_path } /versions/latest'
360+ self .gcp_secret = GcpSecret (version_name )
361+ self .secret_option = f'type:GcpSecret;version_name:{ version_name } '
322362
323363 def tearDown (self ):
324364 if secretmanager is not None :
@@ -334,6 +374,20 @@ def test_gbek_fake_secret_manager_roundtrips(self):
334374 assert_that (
335375 result , equal_to ([('a' , ([1 , 2 ])), ('b' , ([3 ])), ('c' , ([4 ]))]))
336376
377+ @unittest .skipIf (secretmanager is None , 'GCP dependencies are not installed' )
378+ def test_gbk_with_gbek_option_fake_secret_manager_roundtrips (self ):
379+ options = PipelineOptions ()
380+ options .view_as (SetupOptions ).gbek = self .secret_option
381+
382+ with beam .Pipeline (options = options ) as pipeline :
383+ pcoll_1 = pipeline | 'Start 1' >> beam .Create ([('a' , 1 ), ('a' , 2 ),
384+ ('b' , 3 ), ('c' , 4 )])
385+ result = (pcoll_1 ) | beam .GroupByKey ()
386+ sorted_result = result | beam .Map (lambda x : (x [0 ], sorted (x [1 ])))
387+ assert_that (
388+ sorted_result ,
389+ equal_to ([('a' , ([1 , 2 ])), ('b' , ([3 ])), ('c' , ([4 ]))]))
390+
337391 @mock .patch ('apache_beam.transforms.util._DecryptMessage' , MockNoOpDecrypt )
338392 def test_gbek_fake_secret_manager_actually_does_encryption (self ):
339393 fakeSecret = FakeSecret ()
@@ -345,8 +399,23 @@ def test_gbek_fake_secret_manager_actually_does_encryption(self):
345399 assert_that (
346400 result , equal_to ([('a' , ([1 , 2 ])), ('b' , ([3 ])), ('c' , ([4 ]))]))
347401
402+ @mock .patch ('apache_beam.transforms.util._DecryptMessage' , MockNoOpDecrypt )
403+ @mock .patch ('apache_beam.transforms.util.GcpSecret' , FakeSecret )
404+ def test_gbk_actually_does_encryption (self ):
405+ options = PipelineOptions ()
406+ # Version of GcpSecret doesn't matter since it is replaced by FakeSecret
407+ options .view_as (SetupOptions ).gbek = 'type:GcpSecret;version_name:Foo'
408+
409+ with TestPipeline ('FnApiRunner' , options = options ) as pipeline :
410+ pcoll_1 = pipeline | 'Start 1' >> beam .Create ([('a' , 1 ), ('a' , 2 ),
411+ ('b' , 3 ), ('c' , 4 )],
412+ reshuffle = False )
413+ result = pcoll_1 | beam .GroupByKey ()
414+ assert_that (
415+ result , equal_to ([('a' , ([1 , 2 ])), ('b' , ([3 ])), ('c' , ([4 ]))]))
416+
348417 def test_gbek_fake_secret_manager_throws (self ):
349- fakeSecret = FakeSecret (True )
418+ fakeSecret = FakeSecret (None , True )
350419
351420 with self .assertRaisesRegex (RuntimeError , r'Exception retrieving secret' ):
352421 with TestPipeline () as pipeline :
0 commit comments