def test_match_type_variables(self): S = typehints.TypeVariable('S') # pylint: disable=invalid-name T = typehints.TypeVariable('T') # pylint: disable=invalid-name hint = typehints.Dict[S, T] self.assertEqual({ S: int, T: str }, hint.match_type_variables(typehints.Dict[int, str]))
def default_type_hints(self): hints = self._combine_fn.get_type_hints().copy() K = typehints.TypeVariable('K') hints.set_input_types(typehints.Tuple[K, typehints.Any]) if hints.output_types: main_output_type = hints.simple_output_type('') hints.set_output_types(typehints.Tuple[K, main_output_type]) return hints
def default_type_hints(self): hints = self._combine_fn.get_type_hints().copy() K = typehints.TypeVariable('K') if hints.input_types: args, kwargs = hints.input_types args = (typehints.Tuple[K, args[0]],) + args[1:] hints.set_input_types(*args, **kwargs) else: hints.set_input_types(typehints.Tuple[K, typehints.Any]) hints.set_output_types(typehints.Tuple[K, typehints.Any]) return hints
class GroupIntoBatches(PTransform): """PTransform that batches the input into desired batch size. Elements are buffered until they are equal to batch size provided in the argument at which point they are output to the output Pcollection. Windows are preserved (batches will contain elements from the same window) GroupIntoBatches is experimental. Its use case will depend on the runner if it has support of States and Timers. """ def __init__(self, batch_size, max_buffering_duration_secs=None, clock=time.time): """Create a new GroupIntoBatches. Arguments: batch_size: (required) How many elements should be in a batch max_buffering_duration_secs: (optional) How long in seconds at most an incomplete batch of elements is allowed to be buffered in the states. The duration must be a positive second duration and should be given as an int or float. Setting this parameter to zero effectively means no buffering limit. clock: (optional) an alternative to time.time (mostly for testing) """ self.params = _GroupIntoBatchesParams(batch_size, max_buffering_duration_secs) self.clock = clock def expand(self, pcoll): input_coder = coders.registry.get_coder(pcoll) return pcoll | ParDo( _pardo_group_into_batches(input_coder, self.params.batch_size, self.params.max_buffering_duration_secs, self.clock)) def to_runner_api_parameter( self, unused_context # type: PipelineContext ): # type: (...) -> Tuple[str, beam_runner_api_pb2.GroupIntoBatchesPayload] return ( common_urns.group_into_batches_components.GROUP_INTO_BATCHES.urn, self.params.get_payload()) @staticmethod @PTransform.register_urn( common_urns.group_into_batches_components.GROUP_INTO_BATCHES.urn, beam_runner_api_pb2.GroupIntoBatchesPayload) def from_runner_api_parameter(unused_ptransform, proto, unused_context): return GroupIntoBatches(*_GroupIntoBatchesParams.parse_payload(proto)) @typehints.with_input_types(Tuple[K, V]) @typehints.with_output_types(typehints.Tuple[ ShardedKeyType[typehints.TypeVariable(K)], # type: ignore[misc] typehints.Iterable[typehints.TypeVariable(V)]]) class WithShardedKey(PTransform): """A GroupIntoBatches transform that outputs batched elements associated with sharded input keys. By default, keys are sharded to such that the input elements with the same key are spread to all available threads executing the transform. Runners may override the default sharding to do a better load balancing during the execution time. """ def __init__(self, batch_size, max_buffering_duration_secs=None): """Create a new GroupIntoBatches with sharded output. See ``GroupIntoBatches`` transform for a description of input parameters. """ self.params = _GroupIntoBatchesParams(batch_size, max_buffering_duration_secs) _shard_id_prefix = uuid.uuid4().bytes def expand(self, pcoll): key_type, value_type = pcoll.element_type.tuple_types sharded_pcoll = pcoll | Map(lambda key_value: ( ShardedKey( key_value[0], # Use [uuid, thread id] as the shard id. GroupIntoBatches.WithShardedKey._shard_id_prefix + bytes( threading.get_ident().to_bytes(8, 'big'))), key_value[1])).with_output_types(typehints.Tuple[ ShardedKeyType[key_type], # type: ignore[misc] value_type]) return (sharded_pcoll | GroupIntoBatches(self.params.batch_size, self.params.max_buffering_duration_secs)) def to_runner_api_parameter( self, unused_context # type: PipelineContext ): # type: (...) -> Tuple[str, beam_runner_api_pb2.GroupIntoBatchesPayload] return ( common_urns.composites.GROUP_INTO_BATCHES_WITH_SHARDED_KEY.urn, self.params.get_payload()) @staticmethod @PTransform.register_urn( common_urns.composites.GROUP_INTO_BATCHES_WITH_SHARDED_KEY.urn, beam_runner_api_pb2.GroupIntoBatchesPayload) def from_runner_api_parameter(unused_ptransform, proto, unused_context): return GroupIntoBatches.WithShardedKey( *_GroupIntoBatchesParams.parse_payload(proto))
from apache_beam.runners.direct.bundle_factory import BundleFactory from apache_beam.runners.direct.clock import RealClock from apache_beam.runners.direct.clock import TestClock from apache_beam.runners.runner import PipelineResult from apache_beam.runners.runner import PipelineRunner from apache_beam.runners.runner import PipelineState from apache_beam.runners.runner import PValueCache from apache_beam.transforms.core import _GroupAlsoByWindow from apache_beam.transforms.core import _GroupByKeyOnly from apache_beam.transforms.ptransform import PTransform __all__ = ['DirectRunner'] # Type variables. K = typehints.TypeVariable('K') V = typehints.TypeVariable('V') @typehints.with_input_types(typehints.KV[K, V]) @typehints.with_output_types(typehints.KV[K, typehints.Iterable[V]]) class _StreamingGroupByKeyOnly(_GroupByKeyOnly): """Streaming GroupByKeyOnly placeholder for overriding in DirectRunner.""" urn = "direct_runner:streaming_gbko:v0.1" # These are needed due to apply overloads. def to_runner_api_parameter(self, unused_context): return _StreamingGroupByKeyOnly.urn, None @PTransform.register_urn(urn, None) def from_runner_api_parameter(unused_payload, unused_context):
from apache_beam.transforms.core import Map from apache_beam.transforms.core import ParDo from apache_beam.transforms.ptransform import PTransform from apache_beam.transforms.ptransform import ptransform_fn from apache_beam.utils import windowed_value __all__ = [ 'BatchElements', 'CoGroupByKey', 'Keys', 'KvSwap', 'RemoveDuplicates', 'Values', ] T = typehints.TypeVariable('T') class CoGroupByKey(PTransform): """Groups results across several PCollections by key. Given an input dict mapping serializable keys (called "tags") to 0 or more PCollections of (key, value) tuples, e.g.:: {'pc1': pcoll1, 'pc2': pcoll2, 33333: pcoll3} 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::