-
Notifications
You must be signed in to change notification settings - Fork 4.5k
Expand file tree
/
Copy pathutil.py
More file actions
2364 lines (2026 loc) · 84.5 KB
/
util.py
File metadata and controls
2364 lines (2026 loc) · 84.5 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
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
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
#
# 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.
#
"""Simple utility PTransforms.
"""
# pytype: skip-file
import bisect
import collections
import contextlib
import hashlib
import hmac
import logging
import random
import re
import threading
import time
import uuid
from collections.abc import Callable
from collections.abc import Iterable
from typing import TYPE_CHECKING
from typing import Any
from typing import List
from typing import Optional
from typing import Tuple
from typing import TypeVar
from typing import Union
from cryptography.fernet import Fernet
import apache_beam as beam
from apache_beam import coders
from apache_beam import pvalue
from apache_beam import typehints
from apache_beam.metrics import Metrics
from apache_beam.portability import common_urns
from apache_beam.portability.api import beam_runner_api_pb2
from apache_beam.pvalue import AsSideInput
from apache_beam.pvalue import PCollection
from apache_beam.transforms import window
from apache_beam.transforms.combiners import CountCombineFn
from apache_beam.transforms.core import CombinePerKey
from apache_beam.transforms.core import Create
from apache_beam.transforms.core import DoFn
from apache_beam.transforms.core import FlatMap
from apache_beam.transforms.core import Flatten
from apache_beam.transforms.core import GroupByKey
from apache_beam.transforms.core import Map
from apache_beam.transforms.core import MapTuple
from apache_beam.transforms.core import ParDo
from apache_beam.transforms.core import Windowing
from apache_beam.transforms.ptransform import PTransform
from apache_beam.transforms.ptransform import ptransform_fn
from apache_beam.transforms.timeutil import TimeDomain
from apache_beam.transforms.trigger import AccumulationMode
from apache_beam.transforms.trigger import Always
from apache_beam.transforms.userstate import BagStateSpec
from apache_beam.transforms.userstate import CombiningValueStateSpec
from apache_beam.transforms.userstate import ReadModifyWriteStateSpec
from apache_beam.transforms.userstate import TimerSpec
from apache_beam.transforms.userstate import on_timer
from apache_beam.transforms.window import NonMergingWindowFn
from apache_beam.transforms.window import TimestampCombiner
from apache_beam.transforms.window import TimestampedValue
from apache_beam.typehints import trivial_inference
from apache_beam.typehints.decorators import get_signature
from apache_beam.typehints.native_type_compatibility import TypedWindowedValue
from apache_beam.typehints.sharded_key_type import ShardedKeyType
from apache_beam.utils import shared
from apache_beam.utils import windowed_value
from apache_beam.utils.annotations import deprecated
from apache_beam.utils.sharded_key import ShardedKey
from apache_beam.utils.timestamp import Timestamp
if TYPE_CHECKING:
from apache_beam.runners.pipeline_context import PipelineContext
_LOGGER = logging.getLogger(__name__)
__all__ = [
'BatchElements',
'CoGroupByKey',
'Distinct',
'GcpSecret',
'GroupByEncryptedKey',
'Keys',
'KvSwap',
'LogElements',
'Regex',
'Reify',
'RemoveDuplicates',
'Reshuffle',
'Secret',
'ToString',
'Tee',
'Values',
'WithKeys',
'GroupIntoBatches',
'WaitOn'
]
K = TypeVar('K')
V = TypeVar('V')
T = TypeVar('T')
U = TypeVar('U')
RESHUFFLE_TYPEHINT_BREAKING_CHANGE_VERSION = "2.64.0"
class CoGroupByKey(PTransform):
"""Groups results across several PCollections by key.
Given an input dict of serializable keys (called "tags") to 0 or more
PCollections of (key, value) tuples, it creates a single output PCollection
of (key, value) tuples whose keys are the unique input keys from all inputs,
and whose values are dicts mapping each tag to an iterable of whatever values
were under the key in the corresponding PCollection, in this manner::
('some key', {'tag1': ['value 1 under "some key" in pcoll1',
'value 2 under "some key" in pcoll1',
...],
'tag2': ... ,
... })
where `[]` refers to an iterable, not a list.
For example, given::
{'tag1': pc1, 'tag2': pc2, 333: pc3}
where::
pc1 = beam.Create([(k1, v1)]))
pc2 = beam.Create([])
pc3 = beam.Create([(k1, v31), (k1, v32), (k2, v33)])
The output PCollection would consist of items::
[(k1, {'tag1': [v1], 'tag2': [], 333: [v31, v32]}),
(k2, {'tag1': [], 'tag2': [], 333: [v33]})]
where `[]` refers to an iterable, not a list.
CoGroupByKey also works for tuples, lists, or other flat iterables of
PCollections, in which case the values of the resulting PCollections
will be tuples whose nth value is the iterable of values from the nth
PCollection---conceptually, the "tags" are the indices into the input.
Thus, for this input::
(pc1, pc2, pc3)
the output would be::
[(k1, ([v1], [], [v31, v32]),
(k2, ([], [], [v33]))]
where, again, `[]` refers to an iterable, not a list.
Attributes:
**kwargs: Accepts a single named argument "pipeline", which specifies the
pipeline that "owns" this PTransform. Ordinarily CoGroupByKey can obtain
this information from one of the input PCollections, but if there are none
(or if there's a chance there may be none), this argument is the only way
to provide pipeline information, and should be considered mandatory.
"""
def __init__(self, *, pipeline=None):
self.pipeline = pipeline
def _extract_input_pvalues(self, pvalueish):
try:
# If this works, it's a dict.
return pvalueish, tuple(pvalueish.values())
except AttributeError:
# Cast iterables a tuple so we can do re-iteration.
pcolls = tuple(pvalueish)
return pcolls, pcolls
def expand(self, pcolls):
if not pcolls:
pcolls = (self.pipeline | Create([]), )
if isinstance(pcolls, dict):
tags = list(pcolls.keys())
if all(isinstance(tag, str) and len(tag) < 10 for tag in tags):
# Small, string tags. Pass them as data.
pcolls_dict = pcolls
restore_tags = None
else:
# Pass the tags in the restore_tags closure.
tags = list(pcolls.keys())
pcolls_dict = {str(ix): pcolls[tag] for (ix, tag) in enumerate(tags)}
restore_tags = lambda vs: {
tag: vs[str(ix)]
for (ix, tag) in enumerate(tags)
}
else:
# Tags are tuple indices.
tags = [str(ix) for ix in range(len(pcolls))]
pcolls_dict = dict(zip(tags, pcolls))
restore_tags = lambda vs: tuple(vs[tag] for tag in tags)
input_key_types = []
input_value_types = []
for pcoll in pcolls_dict.values():
key_type, value_type = typehints.trivial_inference.key_value_types(
pcoll.element_type)
input_key_types.append(key_type)
input_value_types.append(value_type)
output_key_type = typehints.Union[tuple(input_key_types)]
iterable_input_value_types = tuple(
typehints.Iterable[t] for t in input_value_types)
output_value_type = typehints.Dict[
str, typehints.Union[iterable_input_value_types or [typehints.Any]]]
result = (
pcolls_dict
| 'CoGroupByKeyImpl' >>
_CoGBKImpl(pipeline=self.pipeline).with_output_types(
typehints.Tuple[output_key_type, output_value_type]))
if restore_tags:
if isinstance(pcolls, dict):
dict_key_type = typehints.Union[tuple(
trivial_inference.instance_to_type(tag) for tag in tags)]
output_value_type = typehints.Dict[
dict_key_type, typehints.Union[iterable_input_value_types]]
else:
output_value_type = typehints.Tuple[iterable_input_value_types]
result |= 'RestoreTags' >> MapTuple(
lambda k, vs: (k, restore_tags(vs))).with_output_types(
typehints.Tuple[output_key_type, output_value_type])
return result
class _CoGBKImpl(PTransform):
def __init__(self, *, pipeline=None):
self.pipeline = pipeline
def expand(self, pcolls):
# Check input PCollections for PCollection-ness, and that they all belong
# to the same pipeline.
for pcoll in pcolls.values():
self._check_pcollection(pcoll)
if self.pipeline:
assert pcoll.pipeline == self.pipeline, (
'All input PCollections must belong to the same pipeline.')
tags = list(pcolls.keys())
def add_tag(tag):
return lambda k, v: (k, (tag, v))
def collect_values(key, tagged_values):
grouped_values = {tag: [] for tag in tags}
for tag, value in tagged_values:
grouped_values[tag].append(value)
return key, grouped_values
return ([
pcoll
| 'Tag[%s]' % tag >> MapTuple(add_tag(tag))
for (tag, pcoll) in pcolls.items()
]
| Flatten(pipeline=self.pipeline)
| GroupByKey()
| MapTuple(collect_values).with_input_types(
tuple[K, Iterable[tuple[U, V]]]).with_output_types(
tuple[K, dict[U, list[V]]]))
@ptransform_fn
@typehints.with_input_types(tuple[K, V])
@typehints.with_output_types(K)
def Keys(pcoll, label='Keys'): # pylint: disable=invalid-name
"""Produces a PCollection of first elements of 2-tuples in a PCollection."""
return pcoll | label >> MapTuple(lambda k, _: k)
@ptransform_fn
@typehints.with_input_types(tuple[K, V])
@typehints.with_output_types(V)
def Values(pcoll, label='Values'): # pylint: disable=invalid-name
"""Produces a PCollection of second elements of 2-tuples in a PCollection."""
return pcoll | label >> MapTuple(lambda _, v: v)
@ptransform_fn
@typehints.with_input_types(tuple[K, V])
@typehints.with_output_types(tuple[V, K])
def KvSwap(pcoll, label='KvSwap'): # pylint: disable=invalid-name
"""Produces a PCollection reversing 2-tuples in a PCollection."""
return pcoll | label >> MapTuple(lambda k, v: (v, k))
@ptransform_fn
@typehints.with_input_types(T)
@typehints.with_output_types(T)
def Distinct(pcoll): # pylint: disable=invalid-name
"""Produces a PCollection containing distinct elements of a PCollection."""
return (
pcoll
| 'ToPairs' >> Map(lambda v: (v, None))
| 'Group' >> CombinePerKey(lambda vs: None)
| 'Distinct' >> Keys())
@deprecated(since='2.12', current='Distinct')
@ptransform_fn
@typehints.with_input_types(T)
@typehints.with_output_types(T)
def RemoveDuplicates(pcoll):
"""Produces a PCollection containing distinct elements of a PCollection."""
return pcoll | 'RemoveDuplicates' >> Distinct()
class Secret():
"""A secret management class used for handling sensitive data.
This class provides a generic interface for secret management. Implementations
of this class should handle fetching secrets from a secret management system.
"""
def get_secret_bytes(self) -> bytes:
"""Returns the secret as a byte string."""
raise NotImplementedError()
@staticmethod
def generate_secret_bytes() -> bytes:
"""Generates a new secret key."""
return Fernet.generate_key()
@staticmethod
def parse_secret_option(secret) -> 'Secret':
"""Parses a secret string and returns the appropriate secret type.
The secret string should be formatted like:
'type:<secret_type>;<secret_param>:<value>'
For example, 'type:GcpSecret;version_name:my_secret/versions/latest'
would return a GcpSecret initialized with 'my_secret/versions/latest'.
"""
param_map = {}
for param in secret.split(';'):
parts = param.split(':')
param_map[parts[0]] = parts[1]
if 'type' not in param_map:
raise ValueError('Secret string must contain a valid type parameter')
secret_type = param_map['type'].lower()
del param_map['type']
secret_class = Secret
secret_params = None
if secret_type == 'gcpsecret':
secret_class = GcpSecret # type: ignore[assignment]
secret_params = ['version_name']
elif secret_type == 'gcphsmgeneratedsecret':
secret_class = GcpHsmGeneratedSecret # type: ignore[assignment]
secret_params = [
'project_id', 'location_id', 'key_ring_id', 'key_id', 'job_name'
]
else:
raise ValueError(
f'Invalid secret type {secret_type}, currently only '
'GcpSecret and GcpHsmGeneratedSecret are supported')
for param_name in param_map.keys():
if param_name not in secret_params:
raise ValueError(
f'Invalid secret parameter {param_name}, '
f'{secret_type} only supports the following '
f'parameters: {secret_params}')
return secret_class(**param_map)
class GcpSecret(Secret):
"""A secret manager implementation that retrieves secrets from Google Cloud
Secret Manager.
"""
def __init__(self, version_name: str):
"""Initializes a GcpSecret object.
Args:
version_name: The full version name of the secret in Google Cloud Secret
Manager. For example:
projects/<id>/secrets/<secret_name>/versions/1.
For more info, see
https://cloud.google.com/python/docs/reference/secretmanager/latest/google.cloud.secretmanager_v1beta1.services.secret_manager_service.SecretManagerServiceClient#google_cloud_secretmanager_v1beta1_services_secret_manager_service_SecretManagerServiceClient_access_secret_version
"""
self._version_name = version_name
def get_secret_bytes(self) -> bytes:
try:
from google.cloud import secretmanager
client = secretmanager.SecretManagerServiceClient()
response = client.access_secret_version(
request={"name": self._version_name})
secret = response.payload.data
return secret
except Exception as e:
raise RuntimeError(
'Failed to retrieve secret bytes for secret '
f'{self._version_name} with exception {e}')
def __eq__(self, secret):
return self._version_name == getattr(secret, '_version_name', None)
class GcpHsmGeneratedSecret(Secret):
"""A secret manager implementation that generates a secret using a GCP HSM key
and stores it in Google Cloud Secret Manager. If the secret already exists,
it will be retrieved.
"""
def __init__(
self,
project_id: str,
location_id: str,
key_ring_id: str,
key_id: str,
job_name: str):
"""Initializes a GcpHsmGeneratedSecret object.
Args:
project_id: The GCP project ID.
location_id: The GCP location ID for the HSM key.
key_ring_id: The ID of the KMS key ring.
key_id: The ID of the KMS key.
job_name: The name of the job, used to generate a unique secret name.
"""
self._project_id = project_id
self._location_id = location_id
self._key_ring_id = key_ring_id
self._key_id = key_id
self._secret_version_name = f'HsmGeneratedSecret_{job_name}'
def get_secret_bytes(self) -> bytes:
"""Retrieves the secret bytes.
If the secret version already exists in Secret Manager, it is retrieved.
Otherwise, a new secret and version are created. The new secret is
generated using the HSM key.
Returns:
The secret as a byte string.
"""
try:
from google.api_core import exceptions as api_exceptions
from google.cloud import secretmanager
client = secretmanager.SecretManagerServiceClient()
project_path = f"projects/{self._project_id}"
secret_path = f"{project_path}/secrets/{self._secret_version_name}"
# Since we may generate multiple versions when doing this on workers,
# just always take the first version added to maintain consistency.
secret_version_path = f"{secret_path}/versions/1"
try:
response = client.access_secret_version(
request={"name": secret_version_path})
return response.payload.data
except api_exceptions.NotFound:
# Don't bother logging yet, we'll only log if we actually add the
# secret version below
pass
try:
client.create_secret(
request={
"parent": project_path,
"secret_id": self._secret_version_name,
"secret": {
"replication": {
"automatic": {}
}
},
})
except api_exceptions.AlreadyExists:
# Don't bother logging yet, we'll only log if we actually add the
# secret version below
pass
new_key = self.generate_dek()
try:
# Try one more time in case it was created while we were generating the
# DEK.
response = client.access_secret_version(
request={"name": secret_version_path})
return response.payload.data
except api_exceptions.NotFound:
_LOGGER.info(
"Secret version %s not found. "
"Creating new secret and version.",
secret_version_path)
client.add_secret_version(
request={
"parent": secret_path, "payload": {
"data": new_key
}
})
response = client.access_secret_version(
request={"name": secret_version_path})
return response.payload.data
except Exception as e:
raise RuntimeError(
f'Failed to retrieve or create secret bytes for secret '
f'{self._secret_version_name} with exception {e}')
def generate_dek(self, dek_size: int = 32) -> bytes:
"""Generates a new Data Encryption Key (DEK) using an HSM-backed key.
This function follows a key derivation process that incorporates entropy
from the HSM-backed key into the nonce used for key derivation.
Args:
dek_size: The size of the DEK to generate.
Returns:
A new DEK of the specified size, url-safe base64-encoded.
"""
try:
import base64
import os
from cryptography.hazmat.primitives import hashes
from cryptography.hazmat.primitives.kdf.hkdf import HKDF
from google.cloud import kms
# 1. Generate a random nonce (nonce_one)
nonce_one = os.urandom(dek_size)
# 2. Use the HSM-backed key to encrypt nonce_one to create nonce_two
kms_client = kms.KeyManagementServiceClient()
key_path = kms_client.crypto_key_path(
self._project_id, self._location_id, self._key_ring_id, self._key_id)
response = kms_client.encrypt(
request={
'name': key_path, 'plaintext': nonce_one
})
nonce_two = response.ciphertext
# 3. Generate a Derivation Key (DK)
dk = os.urandom(dek_size)
# 4. Use a KDF to derive the DEK using DK and nonce_two
hkdf = HKDF(
algorithm=hashes.SHA256(),
length=dek_size,
salt=nonce_two,
info=None,
)
dek = hkdf.derive(dk)
return base64.urlsafe_b64encode(dek)
except Exception as e:
raise RuntimeError(f'Failed to generate DEK with exception {e}')
class _EncryptMessage(DoFn):
"""A DoFn that encrypts the key and value of each element."""
def __init__(
self,
hmac_key_secret: Secret,
key_coder: coders.Coder,
value_coder: coders.Coder):
self.hmac_key_secret = hmac_key_secret
self.key_coder = key_coder
self.value_coder = value_coder
def setup(self):
self._hmac_key = self.hmac_key_secret.get_secret_bytes()
self.fernet = Fernet(self._hmac_key)
def process(self,
element: Any) -> Iterable[Tuple[bytes, Tuple[bytes, bytes]]]:
"""Encrypts the key and value of an element.
Args:
element: A tuple containing the key and value to be encrypted.
Yields:
A tuple containing the HMAC of the encoded key, and a tuple of the
encrypted key and value.
"""
k, v = element
encoded_key = self.key_coder.encode(k)
encoded_value = self.value_coder.encode(v)
hmac_encoded_key = hmac.new(self._hmac_key, encoded_key,
hashlib.sha256).digest()
out_element = (
hmac_encoded_key,
(self.fernet.encrypt(encoded_key), self.fernet.encrypt(encoded_value)))
yield out_element
class _DecryptMessage(DoFn):
"""A DoFn that decrypts the key and value of each element."""
def __init__(
self,
hmac_key_secret: Secret,
key_coder: coders.Coder,
value_coder: coders.Coder):
self.hmac_key_secret = hmac_key_secret
self.key_coder = key_coder
self.value_coder = value_coder
def setup(self):
hmac_key = self.hmac_key_secret.get_secret_bytes()
self.fernet = Fernet(hmac_key)
def decode_value(self, encoded_element: Tuple[bytes, bytes]) -> Any:
encrypted_value = encoded_element[1]
encoded_value = self.fernet.decrypt(encrypted_value)
real_val = self.value_coder.decode(encoded_value)
return real_val
def filter_elements_by_key(
self,
encrypted_key: bytes,
encoded_elements: Iterable[Tuple[bytes, bytes]]) -> Iterable[Any]:
for e in encoded_elements:
if encrypted_key == self.fernet.decrypt(e[0]):
yield self.decode_value(e)
# Right now, GBK always returns a list of elements, so we match this behavior
# here. This does mean that the whole list will be materialized every time,
# but passing an Iterable containing an Iterable breaks when pickling happens
def process(
self, element: Tuple[bytes, Iterable[Tuple[bytes, bytes]]]
) -> Iterable[Tuple[Any, List[Any]]]:
"""Decrypts the key and values of an element.
Args:
element: A tuple containing the HMAC of the encoded key and an iterable
of tuples of encrypted keys and values.
Yields:
A tuple containing the decrypted key and a list of decrypted values.
"""
unused_hmac_encoded_key, encoded_elements = element
seen_keys = set()
# Since there could be hmac collisions, we will use the fernet encrypted
# key to confirm that the mapping is actually correct.
for e in encoded_elements:
encrypted_key, unused_encrypted_value = e
encoded_key = self.fernet.decrypt(encrypted_key)
if encoded_key in seen_keys:
continue
seen_keys.add(encoded_key)
real_key = self.key_coder.decode(encoded_key)
yield (
real_key,
list(self.filter_elements_by_key(encoded_key, encoded_elements)))
@typehints.with_input_types(Tuple[K, V])
@typehints.with_output_types(Tuple[K, Iterable[V]])
class GroupByEncryptedKey(PTransform):
"""A PTransform that provides a secure alternative to GroupByKey.
This transform encrypts the keys of the input PCollection, performs a
GroupByKey on the encrypted keys, and then decrypts the keys in the output.
This is useful when the keys contain sensitive data that should not be
stored at rest by the runner. Note the following caveats:
1) Runners can implement arbitrary materialization steps, so this does not
guarantee that the whole pipeline will not have unencrypted data at rest by
itself.
2) If using this transform in streaming mode, this transform may not properly
handle update compatibility checks around coders. This means that an improper
update could lead to invalid coders, causing pipeline failure or data
corruption. If you need to update, make sure that the input type passed into
this transform does not change.
"""
def __init__(self, hmac_key: Secret):
"""Initializes a GroupByEncryptedKey transform.
Args:
hmac_key: A Secret object that provides the secret key for HMAC and
encryption. For example, a GcpSecret can be used to access a secret
stored in GCP Secret Manager
"""
self._hmac_key = hmac_key
def expand(self, pcoll):
key_type, value_type = (typehints.typehints.coerce_to_kv_type(
pcoll.element_type).tuple_types)
kv_type_hint = typehints.KV[key_type, value_type]
if kv_type_hint and kv_type_hint != typehints.Any:
coder = coders.registry.get_coder(kv_type_hint)
try:
coder = coder.as_deterministic_coder(self.label)
except ValueError:
_LOGGER.warning(
'GroupByEncryptedKey %s: '
'The key coder is not deterministic. This may result in incorrect '
'pipeline output. This can be fixed by adding a type hint to the '
'operation preceding the GroupByKey step, and for custom key '
'classes, by writing a deterministic custom Coder. Please see the '
'documentation for more details.',
self.label)
if not coder.is_kv_coder():
raise ValueError(
'Input elements to the transform %s with stateful DoFn must be '
'key-value pairs.' % self)
key_coder = coder.key_coder()
value_coder = coder.value_coder()
else:
key_coder = coders.registry.get_coder(typehints.Any)
value_coder = key_coder
gbk = beam.GroupByKey()
gbk._inside_gbek = True
output_type = Tuple[key_type, Iterable[value_type]]
return (
pcoll
| beam.ParDo(_EncryptMessage(self._hmac_key, key_coder, value_coder))
| gbk
| beam.ParDo(
_DecryptMessage(self._hmac_key, key_coder,
value_coder)).with_output_types(output_type))
class _BatchSizeEstimator(object):
"""Estimates the best size for batches given historical timing.
"""
_MAX_DATA_POINTS = 100
_MAX_GROWTH_FACTOR = 2
def __init__(
self,
min_batch_size=1,
max_batch_size=10000,
target_batch_overhead=.05,
target_batch_duration_secs=10,
target_batch_duration_secs_including_fixed_cost=None,
variance=0.25,
clock=time.time,
ignore_first_n_seen_per_batch_size=0,
record_metrics=True):
if min_batch_size > max_batch_size:
raise ValueError(
"Minimum (%s) must not be greater than maximum (%s)" %
(min_batch_size, max_batch_size))
if target_batch_overhead and not 0 < target_batch_overhead <= 1:
raise ValueError(
"target_batch_overhead (%s) must be between 0 and 1" %
(target_batch_overhead))
if target_batch_duration_secs and target_batch_duration_secs <= 0:
raise ValueError(
"target_batch_duration_secs (%s) must be positive" %
(target_batch_duration_secs))
if (target_batch_duration_secs_including_fixed_cost and
target_batch_duration_secs_including_fixed_cost <= 0):
raise ValueError(
"target_batch_duration_secs_including_fixed_cost "
"(%s) must be positive" %
(target_batch_duration_secs_including_fixed_cost))
if not (target_batch_overhead or target_batch_duration_secs or
target_batch_duration_secs_including_fixed_cost):
raise ValueError(
"At least one of target_batch_overhead or "
"target_batch_duration_secs or "
"target_batch_duration_secs_including_fixed_cost must be positive.")
if ignore_first_n_seen_per_batch_size < 0:
raise ValueError(
'ignore_first_n_seen_per_batch_size (%s) must be non '
'negative' % (ignore_first_n_seen_per_batch_size))
self._min_batch_size = min_batch_size
self._max_batch_size = max_batch_size
self._target_batch_overhead = target_batch_overhead
self._target_batch_duration_secs = target_batch_duration_secs
self._target_batch_duration_secs_including_fixed_cost = (
target_batch_duration_secs_including_fixed_cost)
self._variance = variance
self._clock = clock
self._data = []
self._ignore_next_timing = False
self._ignore_first_n_seen_per_batch_size = (
ignore_first_n_seen_per_batch_size)
self._batch_size_num_seen = {}
self._replay_last_batch_size = None
self._record_metrics = record_metrics
self._element_count = 0
self._batch_count = 0
if record_metrics:
self._size_distribution = Metrics.distribution(
'BatchElements', 'batch_size')
self._time_distribution = Metrics.distribution(
'BatchElements', 'msec_per_batch')
else:
self._size_distribution = self._time_distribution = None
# Beam distributions only accept integer values, so we use this to
# accumulate under-reported values until they add up to whole milliseconds.
# (Milliseconds are chosen because that's conventionally used elsewhere in
# profiling-style counters.)
self._remainder_msecs = 0
def ignore_next_timing(self):
"""Call to indicate the next timing should be ignored.
For example, the first emit of a ParDo operation is known to be anomalous
due to setup that may occur.
"""
self._ignore_next_timing = True
@contextlib.contextmanager
def record_time(self, batch_size):
start = self._clock()
yield
elapsed = float(self._clock() - start)
elapsed_msec = 1e3 * elapsed + self._remainder_msecs
if self._record_metrics:
self._size_distribution.update(batch_size)
self._time_distribution.update(int(elapsed_msec))
self._element_count += batch_size
self._batch_count += 1
self._remainder_msecs = elapsed_msec - int(elapsed_msec)
# If we ignore the next timing, replay the batch size to get accurate
# timing.
if self._ignore_next_timing:
self._ignore_next_timing = False
self._replay_last_batch_size = min(batch_size, self._max_batch_size)
else:
self._data.append((batch_size, elapsed))
if len(self._data) >= self._MAX_DATA_POINTS:
self._thin_data()
def _thin_data(self):
# Make sure we don't change the parity of len(self._data)
# As it's used below to alternate jitter.
self._data.pop(random.randrange(len(self._data) // 4))
self._data.pop(random.randrange(len(self._data) // 2))
@staticmethod
def linear_regression_no_numpy(xs, ys):
# Least squares fit for y = a + bx over all points.
n = float(len(xs))
xbar = sum(xs) / n
ybar = sum(ys) / n
if xbar == 0:
return ybar, 0
if all(xs[0] == x for x in xs):
# Simply use the mean if all values in xs are same.
return 0, ybar / xbar
b = (
sum([(x - xbar) * (y - ybar)
for x, y in zip(xs, ys)]) / sum([(x - xbar)**2 for x in xs]))
a = ybar - b * xbar
return a, b
@staticmethod
def linear_regression_numpy(xs, ys):
# pylint: disable=wrong-import-order, wrong-import-position
import numpy as np
from numpy import sum
n = len(xs)
if all(xs[0] == x for x in xs):
# If all values of xs are same then fallback to linear_regression_no_numpy
return _BatchSizeEstimator.linear_regression_no_numpy(xs, ys)
xs = np.asarray(xs, dtype=float)
ys = np.asarray(ys, dtype=float)
# First do a simple least squares fit for y = a + bx over all points.
b, a = np.polyfit(xs, ys, 1)
if n < 10:
return a, b
else:
# Refine this by throwing out outliers, according to Cook's distance.
# https://en.wikipedia.org/wiki/Cook%27s_distance
sum_x = sum(xs)
sum_x2 = sum(xs**2)
errs = a + b * xs - ys
s2 = sum(errs**2) / (n - 2)
if s2 == 0:
# It's an exact fit!
return a, b
h = (sum_x2 - 2 * sum_x * xs + n * xs**2) / (n * sum_x2 - sum_x**2)
cook_ds = 0.5 / s2 * errs**2 * (h / (1 - h)**2)
# Re-compute the regression, excluding those points with Cook's distance
# greater than 0.5, and weighting by the inverse of x to give a more
# stable y-intercept (as small batches have relatively more information
# about the fixed overhead).
weight = (cook_ds <= 0.5) / xs
b, a = np.polyfit(xs, ys, 1, w=weight)
return a, b
try:
# pylint: disable=wrong-import-order, wrong-import-position
import numpy as np
linear_regression = linear_regression_numpy
except ImportError:
linear_regression = linear_regression_no_numpy
def _calculate_next_batch_size(self):
if self._min_batch_size == self._max_batch_size:
return self._min_batch_size
elif len(self._data) < 1:
return self._min_batch_size
elif len(self._data) < 2:
# Force some variety so we have distinct batch sizes on which to do
# linear regression below.
return int(
max(
min(
self._max_batch_size,
self._min_batch_size * self._MAX_GROWTH_FACTOR),
self._min_batch_size + 1))
# There tends to be a lot of noise in the top quantile, which also
# has outsided influence in the regression. If we have enough data,
# Simply declare the top 20% to be outliers.
trimmed_data = sorted(self._data)[:max(20, len(self._data) * 4 // 5)]
# Linear regression for y = a + bx, where x is batch size and y is time.
xs, ys = zip(*trimmed_data)
a, b = self.linear_regression(xs, ys)
# Avoid nonsensical or division-by-zero errors below due to noise.
a = max(a, 1e-10)
b = max(b, 1e-20)
last_batch_size = self._data[-1][0]
cap = min(last_batch_size * self._MAX_GROWTH_FACTOR, self._max_batch_size)
target = self._max_batch_size
if self._target_batch_duration_secs_including_fixed_cost:
# Solution to
# a + b*x = self._target_batch_duration_secs_including_fixed_cost.
target = min(
target,
(self._target_batch_duration_secs_including_fixed_cost - a) / b)
if self._target_batch_duration_secs:
# Solution to b*x = self._target_batch_duration_secs.
# We ignore the fixed cost in this computation as it has negligeabel
# impact when it is small and unhelpfully forces the minimum batch size
# when it is large.
target = min(target, self._target_batch_duration_secs / b)
if self._target_batch_overhead:
# Solution to a / (a + b*x) = self._target_batch_overhead.
target = min(target, (a / b) * (1 / self._target_batch_overhead - 1))
# Avoid getting stuck at a single batch size (especially the minimal
# batch size) which would not allow us to extrapolate to other batch
# sizes.
# Jitter alternates between 0 and 1.
jitter = len(self._data) % 2
# Smear our samples across a range centered at the target.
if len(self._data) > 10:
target += int(target * self._variance * 2 * (random.random() - .5))
return int(max(self._min_batch_size + jitter, min(target, cap)))
def next_batch_size(self):
# Check if we should replay a previous batch size due to it not being
# recorded.
if self._replay_last_batch_size:
result = self._replay_last_batch_size
self._replay_last_batch_size = None
else:
result = self._calculate_next_batch_size()
seen_count = self._batch_size_num_seen.get(result, 0) + 1
if seen_count <= self._ignore_first_n_seen_per_batch_size:
self.ignore_next_timing()
self._batch_size_num_seen[result] = seen_count
return result
def stats(self):
return "element_count=%s batch_count=%s next_batch_size=%s timings=%s" % (
self._element_count,
self._batch_count,
self._calculate_next_batch_size(),
self._data)
class _GlobalWindowsBatchingDoFn(DoFn):