github.com/apache/beam/sdks/v2@v2.48.2/python/apache_beam/pvalue.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 """PValue, PCollection: one node of a dataflow graph. 19 20 A node of a dataflow processing graph is a PValue. Currently, there is only 21 one type: PCollection (a potentially very large set of arbitrary values). 22 Once created, a PValue belongs to a pipeline and has an associated 23 transform (of type PTransform), which describes how the value will be 24 produced when the pipeline gets executed. 25 """ 26 27 # pytype: skip-file 28 29 import collections 30 import itertools 31 from typing import TYPE_CHECKING 32 from typing import Any 33 from typing import Dict 34 from typing import Generic 35 from typing import Iterator 36 from typing import Optional 37 from typing import Sequence 38 from typing import TypeVar 39 from typing import Union 40 41 from apache_beam import coders 42 from apache_beam import typehints 43 from apache_beam.internal import pickler 44 from apache_beam.portability import common_urns 45 from apache_beam.portability import python_urns 46 from apache_beam.portability.api import beam_runner_api_pb2 47 48 if TYPE_CHECKING: 49 from apache_beam.transforms import sideinputs 50 from apache_beam.transforms.core import ParDo 51 from apache_beam.transforms.core import Windowing 52 from apache_beam.pipeline import AppliedPTransform 53 from apache_beam.pipeline import Pipeline 54 from apache_beam.runners.pipeline_context import PipelineContext 55 56 __all__ = [ 57 'PCollection', 58 'TaggedOutput', 59 'AsSideInput', 60 'AsSingleton', 61 'AsIter', 62 'AsList', 63 'AsDict', 64 'EmptySideInput', 65 'Row', 66 ] 67 68 T = TypeVar('T') 69 70 71 class PValue(object): 72 """Base class for PCollection. 73 74 Dataflow users should not construct PValue objects directly in their 75 pipelines. 76 77 A PValue has the following main characteristics: 78 (1) Belongs to a pipeline. Added during object initialization. 79 (2) Has a transform that can compute the value if executed. 80 (3) Has a value which is meaningful if the transform was executed. 81 """ 82 83 def __init__(self, 84 pipeline, # type: Pipeline 85 tag=None, # type: Optional[str] 86 element_type=None, # type: Optional[Union[type,typehints.TypeConstraint]] 87 windowing=None, # type: Optional[Windowing] 88 is_bounded=True, 89 ): 90 """Initializes a PValue with all arguments hidden behind keyword arguments. 91 92 Args: 93 pipeline: Pipeline object for this PValue. 94 tag: Tag of this PValue. 95 element_type: The type of this PValue. 96 """ 97 self.pipeline = pipeline 98 self.tag = tag 99 self.element_type = element_type 100 # The AppliedPTransform instance for the application of the PTransform 101 # generating this PValue. The field gets initialized when a transform 102 # gets applied. 103 self.producer = None # type: Optional[AppliedPTransform] 104 self.is_bounded = is_bounded 105 if windowing: 106 self._windowing = windowing 107 self.requires_deterministic_key_coder = None 108 109 def __str__(self): 110 return self._str_internal() 111 112 def __repr__(self): 113 return '<%s at %s>' % (self._str_internal(), hex(id(self))) 114 115 def _str_internal(self): 116 return "%s[%s.%s]" % ( 117 self.__class__.__name__, 118 self.producer.full_label if self.producer else None, 119 self.tag) 120 121 def apply(self, *args, **kwargs): 122 """Applies a transform or callable to a PValue. 123 124 Args: 125 *args: positional arguments. 126 **kwargs: keyword arguments. 127 128 The method will insert the pvalue as the next argument following an 129 optional first label and a transform/callable object. It will call the 130 pipeline.apply() method with this modified argument list. 131 """ 132 arglist = list(args) 133 arglist.insert(1, self) 134 return self.pipeline.apply(*arglist, **kwargs) 135 136 def __or__(self, ptransform): 137 return self.pipeline.apply(ptransform, self) 138 139 140 class PCollection(PValue, Generic[T]): 141 """A multiple values (potentially huge) container. 142 143 Dataflow users should not construct PCollection objects directly in their 144 pipelines. 145 """ 146 def __eq__(self, other): 147 if isinstance(other, PCollection): 148 return self.tag == other.tag and self.producer == other.producer 149 150 def __hash__(self): 151 return hash((self.tag, self.producer)) 152 153 @property 154 def windowing(self): 155 # type: () -> Windowing 156 if not hasattr(self, '_windowing'): 157 assert self.producer is not None and self.producer.transform is not None 158 self._windowing = self.producer.transform.get_windowing( 159 self.producer.inputs) 160 return self._windowing 161 162 def __reduce_ex__(self, unused_version): 163 # Pickling a PCollection is almost always the wrong thing to do, but we 164 # can't prohibit it as it often gets implicitly picked up (e.g. as part 165 # of a closure). 166 return _InvalidUnpickledPCollection, () 167 168 @staticmethod 169 def from_(pcoll, is_bounded=None): 170 # type: (PValue, Optional[bool]) -> PCollection 171 172 """Create a PCollection, using another PCollection as a starting point. 173 174 Transfers relevant attributes. 175 """ 176 if is_bounded is None: 177 is_bounded = pcoll.is_bounded 178 return PCollection(pcoll.pipeline, is_bounded=is_bounded) 179 180 def to_runner_api(self, context): 181 # type: (PipelineContext) -> beam_runner_api_pb2.PCollection 182 return beam_runner_api_pb2.PCollection( 183 unique_name=self._unique_name(), 184 coder_id=context.coder_id_from_element_type( 185 self.element_type, self.requires_deterministic_key_coder), 186 is_bounded=beam_runner_api_pb2.IsBounded.BOUNDED 187 if self.is_bounded else beam_runner_api_pb2.IsBounded.UNBOUNDED, 188 windowing_strategy_id=context.windowing_strategies.get_id( 189 self.windowing)) 190 191 def _unique_name(self): 192 # type: () -> str 193 if self.producer: 194 return '%d%s.%s' % ( 195 len(self.producer.full_label), self.producer.full_label, self.tag) 196 else: 197 return 'PCollection%s' % id(self) 198 199 @staticmethod 200 def from_runner_api(proto, context): 201 # type: (beam_runner_api_pb2.PCollection, PipelineContext) -> PCollection 202 # Producer and tag will be filled in later, the key point is that the same 203 # object is returned for the same pcollection id. 204 # We pass None for the PCollection's Pipeline to avoid a cycle during 205 # deserialization. It will be populated soon after this call, in 206 # Pipeline.from_runner_api(). This brief period is the only time that 207 # PCollection.pipeline is allowed to be None. 208 return PCollection( 209 None, # type: ignore[arg-type] 210 element_type=context.element_type_from_coder_id(proto.coder_id), 211 windowing=context.windowing_strategies.get_by_id( 212 proto.windowing_strategy_id), 213 is_bounded=proto.is_bounded == beam_runner_api_pb2.IsBounded.BOUNDED) 214 215 216 class _InvalidUnpickledPCollection(object): 217 pass 218 219 220 class PBegin(PValue): 221 """A pipeline begin marker used as input to create/read transforms. 222 223 The class is used internally to represent inputs to Create and Read 224 transforms. This allows us to have transforms that uniformly take PValue(s) 225 as inputs. 226 """ 227 pass 228 229 230 class PDone(PValue): 231 """PDone is the output of a transform that has a trivial result such as Write. 232 """ 233 pass 234 235 236 class DoOutputsTuple(object): 237 """An object grouping the multiple outputs of a ParDo or FlatMap transform.""" 238 239 def __init__(self, 240 pipeline, # type: Pipeline 241 transform, # type: ParDo 242 tags, # type: Sequence[str] 243 main_tag, # type: Optional[str] 244 allow_unknown_tags=None, # type: Optional[bool] 245 ): 246 self._pipeline = pipeline 247 self._tags = tags 248 self._main_tag = main_tag 249 self._transform = transform 250 self._allow_unknown_tags = ( 251 not tags if allow_unknown_tags is None else allow_unknown_tags) 252 # The ApplyPTransform instance for the application of the multi FlatMap 253 # generating this value. The field gets initialized when a transform 254 # gets applied. 255 self.producer = None # type: Optional[AppliedPTransform] 256 # Dictionary of PCollections already associated with tags. 257 self._pcolls = {} # type: Dict[Optional[str], PCollection] 258 259 def __str__(self): 260 return '<%s>' % self._str_internal() 261 262 def __repr__(self): 263 return '<%s at %s>' % (self._str_internal(), hex(id(self))) 264 265 def _str_internal(self): 266 return '%s main_tag=%s tags=%s transform=%s' % ( 267 self.__class__.__name__, self._main_tag, self._tags, self._transform) 268 269 def __iter__(self): 270 # type: () -> Iterator[PCollection] 271 272 """Iterates over tags returning for each call a (tag, pcollection) pair.""" 273 if self._main_tag is not None: 274 yield self[self._main_tag] 275 for tag in self._tags: 276 yield self[tag] 277 278 def __getattr__(self, tag): 279 # type: (str) -> PCollection 280 # Special methods which may be accessed before the object is 281 # fully constructed (e.g. in unpickling). 282 if tag[:2] == tag[-2:] == '__': 283 return object.__getattr__(self, tag) # type: ignore 284 return self[tag] 285 286 def __getitem__(self, tag): 287 # type: (Union[int, str, None]) -> PCollection 288 # Accept int tags so that we can look at Partition tags with the 289 # same ints that we used in the partition function. 290 # TODO(gildea): Consider requiring string-based tags everywhere. 291 # This will require a partition function that does not return ints. 292 if isinstance(tag, int): 293 tag = str(tag) 294 if tag == self._main_tag: 295 tag = None 296 elif self._tags and tag not in self._tags and not self._allow_unknown_tags: 297 raise ValueError( 298 "Tag '%s' is neither the main tag '%s' " 299 "nor any of the tags %s" % (tag, self._main_tag, self._tags)) 300 # Check if we accessed this tag before. 301 if tag in self._pcolls: 302 return self._pcolls[tag] 303 304 assert self.producer is not None 305 if tag is not None: 306 self._transform.output_tags.add(tag) 307 pcoll = PCollection(self._pipeline, tag=tag, element_type=typehints.Any) 308 # Transfer the producer from the DoOutputsTuple to the resulting 309 # PCollection. 310 pcoll.producer = self.producer.parts[0] 311 # Add this as an output to both the inner ParDo and the outer _MultiParDo 312 # PTransforms. 313 if tag not in self.producer.parts[0].outputs: 314 self.producer.parts[0].add_output(pcoll, tag) 315 self.producer.add_output(pcoll, tag) 316 else: 317 # Main output is output of inner ParDo. 318 pval = self.producer.parts[0].outputs[None] 319 assert isinstance(pval, 320 PCollection), ("DoOutputsTuple should follow a ParDo.") 321 pcoll = pval 322 self._pcolls[tag] = pcoll 323 return pcoll 324 325 326 class TaggedOutput(object): 327 """An object representing a tagged value. 328 329 ParDo, Map, and FlatMap transforms can emit values on multiple outputs which 330 are distinguished by string tags. The DoFn will return plain values 331 if it wants to emit on the main output and TaggedOutput objects 332 if it wants to emit a value on a specific tagged output. 333 """ 334 def __init__(self, tag, value): 335 # type: (str, Any) -> None 336 if not isinstance(tag, str): 337 raise TypeError( 338 'Attempting to create a TaggedOutput with non-string tag %s' % 339 (tag, )) 340 self.tag = tag 341 self.value = value 342 343 344 class AsSideInput(object): 345 """Marker specifying that a PCollection will be used as a side input. 346 347 When a PCollection is supplied as a side input to a PTransform, it is 348 necessary to indicate how the PCollection should be made available 349 as a PTransform side argument (e.g. in the form of an iterable, mapping, 350 or single value). This class is the superclass of all the various 351 options, and should not be instantiated directly. (See instead AsSingleton, 352 AsIter, etc.) 353 """ 354 def __init__(self, pcoll): 355 # type: (PCollection) -> None 356 from apache_beam.transforms import sideinputs 357 self.pvalue = pcoll 358 self._window_mapping_fn = sideinputs.default_window_mapping_fn( 359 pcoll.windowing.windowfn) 360 361 def _view_options(self): 362 """Internal options corresponding to specific view. 363 364 Intended for internal use by runner implementations. 365 366 Returns: 367 Tuple of options for the given view. 368 """ 369 return { 370 'window_mapping_fn': self._window_mapping_fn, 371 'coder': self._windowed_coder(), 372 } 373 374 @property 375 def element_type(self): 376 return typehints.Any 377 378 def _windowed_coder(self): 379 return coders.WindowedValueCoder( 380 coders.registry.get_coder( 381 self.pvalue.element_type or self.element_type), 382 self.pvalue.windowing.windowfn.get_window_coder()) 383 384 # TODO(robertwb): Get rid of _from_runtime_iterable and _view_options 385 # in favor of _side_input_data(). 386 def _side_input_data(self): 387 # type: () -> SideInputData 388 view_options = self._view_options() 389 from_runtime_iterable = type(self)._from_runtime_iterable 390 return SideInputData( 391 common_urns.side_inputs.ITERABLE.urn, 392 self._window_mapping_fn, 393 lambda iterable: from_runtime_iterable(iterable, view_options)) 394 395 def to_runner_api(self, context): 396 # type: (PipelineContext) -> beam_runner_api_pb2.SideInput 397 return self._side_input_data().to_runner_api(context) 398 399 @staticmethod 400 def from_runner_api(proto, # type: beam_runner_api_pb2.SideInput 401 context # type: PipelineContext 402 ): 403 # type: (...) -> _UnpickledSideInput 404 return _UnpickledSideInput(SideInputData.from_runner_api(proto, context)) 405 406 @staticmethod 407 def _from_runtime_iterable(it, options): 408 raise NotImplementedError 409 410 def requires_keyed_input(self): 411 return False 412 413 414 class _UnpickledSideInput(AsSideInput): 415 def __init__(self, side_input_data): 416 # type: (SideInputData) -> None 417 self._data = side_input_data 418 self._window_mapping_fn = side_input_data.window_mapping_fn 419 420 @staticmethod 421 def _from_runtime_iterable(it, options): 422 access_pattern = options['data'].access_pattern 423 if access_pattern == common_urns.side_inputs.ITERABLE.urn: 424 raw_view = it 425 elif access_pattern == common_urns.side_inputs.MULTIMAP.urn: 426 raw_view = collections.defaultdict(list) 427 for k, v in it: 428 raw_view[k].append(v) 429 else: 430 raise ValueError('Unknown access_pattern: %s' % access_pattern) 431 return options['data'].view_fn(raw_view) 432 433 def _view_options(self): 434 return { 435 'data': self._data, 436 # For non-fn-api runners. 437 'window_mapping_fn': self._data.window_mapping_fn, 438 'coder': self._windowed_coder(), 439 } 440 441 def _side_input_data(self): 442 return self._data 443 444 445 class SideInputData(object): 446 """All of the data about a side input except for the bound PCollection.""" 447 def __init__(self, 448 access_pattern, # type: str 449 window_mapping_fn, # type: sideinputs.WindowMappingFn 450 view_fn 451 ): 452 self.access_pattern = access_pattern 453 self.window_mapping_fn = window_mapping_fn 454 self.view_fn = view_fn 455 456 def to_runner_api(self, context): 457 # type: (PipelineContext) -> beam_runner_api_pb2.SideInput 458 return beam_runner_api_pb2.SideInput( 459 access_pattern=beam_runner_api_pb2.FunctionSpec( 460 urn=self.access_pattern), 461 view_fn=beam_runner_api_pb2.FunctionSpec( 462 urn=python_urns.PICKLED_VIEWFN, 463 payload=pickler.dumps(self.view_fn)), 464 window_mapping_fn=beam_runner_api_pb2.FunctionSpec( 465 urn=python_urns.PICKLED_WINDOW_MAPPING_FN, 466 payload=pickler.dumps(self.window_mapping_fn))) 467 468 @staticmethod 469 def from_runner_api(proto, unused_context): 470 # type: (beam_runner_api_pb2.SideInput, PipelineContext) -> SideInputData 471 assert proto.view_fn.urn == python_urns.PICKLED_VIEWFN 472 assert ( 473 proto.window_mapping_fn.urn == python_urns.PICKLED_WINDOW_MAPPING_FN) 474 return SideInputData( 475 proto.access_pattern.urn, 476 pickler.loads(proto.window_mapping_fn.payload), 477 pickler.loads(proto.view_fn.payload)) 478 479 480 class AsSingleton(AsSideInput): 481 """Marker specifying that an entire PCollection is to be used as a side input. 482 483 When a PCollection is supplied as a side input to a PTransform, it is 484 necessary to indicate whether the entire PCollection should be made available 485 as a PTransform side argument (in the form of an iterable), or whether just 486 one value should be pulled from the PCollection and supplied as the side 487 argument (as an ordinary value). 488 489 Wrapping a PCollection side input argument to a PTransform in this container 490 (e.g., data.apply('label', MyPTransform(), AsSingleton(my_side_input) ) 491 selects the latter behavior. 492 493 The input PCollection must contain exactly one value per window, unless a 494 default is given, in which case it may be empty. 495 """ 496 _NO_DEFAULT = object() 497 498 def __init__(self, pcoll, default_value=_NO_DEFAULT): 499 # type: (PCollection, Any) -> None 500 super().__init__(pcoll) 501 self.default_value = default_value 502 503 def __repr__(self): 504 return 'AsSingleton(%s)' % self.pvalue 505 506 def _view_options(self): 507 base = super()._view_options() 508 if self.default_value != AsSingleton._NO_DEFAULT: 509 return dict(base, default=self.default_value) 510 return base 511 512 @staticmethod 513 def _from_runtime_iterable(it, options): 514 head = list(itertools.islice(it, 2)) 515 if not head: 516 return options.get('default', EmptySideInput()) 517 elif len(head) == 1: 518 return head[0] 519 raise ValueError( 520 'PCollection of size %d with more than one element accessed as a ' 521 'singleton view. First two elements encountered are "%s", "%s".' % 522 (len(head), str(head[0]), str(head[1]))) 523 524 @property 525 def element_type(self): 526 return self.pvalue.element_type 527 528 529 class AsIter(AsSideInput): 530 """Marker specifying that an entire PCollection is to be used as a side input. 531 532 When a PCollection is supplied as a side input to a PTransform, it is 533 necessary to indicate whether the entire PCollection should be made available 534 as a PTransform side argument (in the form of an iterable), or whether just 535 one value should be pulled from the PCollection and supplied as the side 536 argument (as an ordinary value). 537 538 Wrapping a PCollection side input argument to a PTransform in this container 539 (e.g., data.apply('label', MyPTransform(), AsIter(my_side_input) ) selects the 540 former behavor. 541 """ 542 def __repr__(self): 543 return 'AsIter(%s)' % self.pvalue 544 545 @staticmethod 546 def _from_runtime_iterable(it, options): 547 return it 548 549 def _side_input_data(self): 550 # type: () -> SideInputData 551 return SideInputData( 552 common_urns.side_inputs.ITERABLE.urn, 553 self._window_mapping_fn, 554 lambda iterable: iterable) 555 556 @property 557 def element_type(self): 558 return typehints.Iterable[self.pvalue.element_type] 559 560 561 class AsList(AsSideInput): 562 """Marker specifying that an entire PCollection is to be used as a side input. 563 564 Intended for use in side-argument specification---the same places where 565 AsSingleton and AsIter are used, but forces materialization of this 566 PCollection as a list. 567 568 Args: 569 pcoll: Input pcollection. 570 571 Returns: 572 An AsList-wrapper around a PCollection whose one element is a list 573 containing all elements in pcoll. 574 """ 575 @staticmethod 576 def _from_runtime_iterable(it, options): 577 return list(it) 578 579 def _side_input_data(self): 580 # type: () -> SideInputData 581 return SideInputData( 582 common_urns.side_inputs.ITERABLE.urn, self._window_mapping_fn, list) 583 584 585 class AsDict(AsSideInput): 586 """Marker specifying a PCollection to be used as an indexable side input. 587 588 Intended for use in side-argument specification---the same places where 589 AsSingleton and AsIter are used, but returns an interface that allows 590 key lookup. 591 592 Args: 593 pcoll: Input pcollection. All elements should be key-value pairs (i.e. 594 2-tuples) with unique keys. 595 596 Returns: 597 An AsDict-wrapper around a PCollection whose one element is a dict with 598 entries for uniquely-keyed pairs in pcoll. 599 """ 600 @staticmethod 601 def _from_runtime_iterable(it, options): 602 return dict(it) 603 604 def _side_input_data(self): 605 # type: () -> SideInputData 606 return SideInputData( 607 common_urns.side_inputs.ITERABLE.urn, self._window_mapping_fn, dict) 608 609 610 class AsMultiMap(AsSideInput): 611 """Marker specifying a PCollection to be used as an indexable side input. 612 613 Similar to AsDict, but multiple values may be associated per key, and 614 the keys are fetched lazily rather than all having to fit in memory. 615 616 Intended for use in side-argument specification---the same places where 617 AsSingleton and AsIter are used, but returns an interface that allows 618 key lookup. 619 """ 620 @staticmethod 621 def _from_runtime_iterable(it, options): 622 # Legacy implementation. 623 result = collections.defaultdict(list) 624 for k, v in it: 625 result[k].append(v) 626 return result 627 628 def _side_input_data(self): 629 # type: () -> SideInputData 630 return SideInputData( 631 common_urns.side_inputs.MULTIMAP.urn, 632 self._window_mapping_fn, 633 lambda x: x) 634 635 def requires_keyed_input(self): 636 return True 637 638 639 class EmptySideInput(object): 640 """Value indicating when a singleton side input was empty. 641 642 If a PCollection was furnished as a singleton side input to a PTransform, and 643 that PCollection was empty, then this value is supplied to the DoFn in the 644 place where a value from a non-empty PCollection would have gone. This alerts 645 the DoFn that the side input PCollection was empty. Users may want to check 646 whether side input values are EmptySideInput, but they will very likely never 647 want to create new instances of this class themselves. 648 """ 649 pass 650 651 652 class Row(object): 653 """A dynamic schema'd row object. 654 655 This objects attributes are initialized from the keywords passed into its 656 constructor, e.g. Row(x=3, y=4) will create a Row with two attributes x and y. 657 658 More importantly, when a Row object is returned from a `Map`, `FlatMap`, or 659 `DoFn` type inference is able to deduce the schema of the resulting 660 PCollection, e.g. 661 662 pc | beam.Map(lambda x: Row(x=x, y=0.5 * x)) 663 664 when applied to a PCollection of ints will produce a PCollection with schema 665 `(x=int, y=float)`. 666 667 Note that in Beam 2.30.0 and later, Row objects are sensitive to field order. 668 So `Row(x=3, y=4)` is not considered equal to `Row(y=4, x=3)`. 669 """ 670 def __init__(self, **kwargs): 671 self.__dict__.update(kwargs) 672 673 def as_dict(self): 674 return dict(self.__dict__) 675 676 def __iter__(self): 677 for _, value in self.__dict__.items(): 678 yield value 679 680 def __repr__(self): 681 return 'Row(%s)' % ', '.join('%s=%r' % kv for kv in self.__dict__.items()) 682 683 def __hash__(self): 684 return hash(self.__dict__.items()) 685 686 def __eq__(self, other): 687 return ( 688 type(self) == type(other) and 689 len(self.__dict__) == len(other.__dict__) and all( 690 s == o for s, 691 o in zip(self.__dict__.items(), other.__dict__.items()))) 692 693 def __reduce__(self): 694 return _make_Row, tuple(self.__dict__.items()) 695 696 697 def _make_Row(*items): 698 return Row(**dict(items))