def assert_typehints_equal(self, left, right): left = typehints.normalize(left) right = typehints.normalize(right) if match_is_named_tuple(left): self.assertTrue(match_is_named_tuple(right)) self.assertEqual(left.__annotations__, right.__annotations__) else: self.assertEqual(left, right)
def assert_namedtuple_equivalent(self, actual, expected): # Two types are only considered equal if they are literally the same # object (i.e. `actual == expected` is the same as `actual is expected` in # this case). # That's a much stricter check than we need, and it's necessarily not true # if types are pickled/unpickled. Here we just verify the features of the # types that actually matter to us. self.assertTrue(match_is_named_tuple(expected)) self.assertTrue(match_is_named_tuple(actual)) self.assertEqual(actual.__annotations__, expected.__annotations__) self.assertEqual(dir(actual), dir(expected))
def typing_to_runner_api(type_): if match_is_named_tuple(type_): schema = None if hasattr(type_, _BEAM_SCHEMA_ID): schema = SCHEMA_REGISTRY.get_schema_by_id(getattr(type_, _BEAM_SCHEMA_ID)) if schema is None: fields = [ schema_pb2.Field( name=name, type=typing_to_runner_api(type_._field_types[name])) for name in type_._fields ] type_id = str(uuid4()) schema = schema_pb2.Schema(fields=fields, id=type_id) setattr(type_, _BEAM_SCHEMA_ID, type_id) SCHEMA_REGISTRY.add(type_, schema) return schema_pb2.FieldType(row_type=schema_pb2.RowType(schema=schema)) # All concrete types (other than NamedTuple sub-classes) should map to # a supported primitive type. elif type_ in PRIMITIVE_TO_ATOMIC_TYPE: return schema_pb2.FieldType(atomic_type=PRIMITIVE_TO_ATOMIC_TYPE[type_]) elif _match_is_exactly_mapping(type_): key_type, value_type = map(typing_to_runner_api, _get_args(type_)) return schema_pb2.FieldType( map_type=schema_pb2.MapType(key_type=key_type, value_type=value_type)) elif _match_is_optional(type_): # It's possible that a user passes us Optional[Optional[T]], but in python # typing this is indistinguishable from Optional[T] - both resolve to # Union[T, None] - so there's no need to check for that case here. result = typing_to_runner_api(extract_optional_type(type_)) result.nullable = True return result elif _safe_issubclass(type_, Sequence): element_type = typing_to_runner_api(_get_args(type_)[0]) return schema_pb2.FieldType( array_type=schema_pb2.ArrayType(element_type=element_type)) elif _safe_issubclass(type_, Mapping): key_type, value_type = map(typing_to_runner_api, _get_args(type_)) return schema_pb2.FieldType( map_type=schema_pb2.MapType(key_type=key_type, value_type=value_type)) try: logical_type = LogicalType.from_typing(type_) except ValueError: # Unknown type, just treat it like Any return schema_pb2.FieldType( logical_type=schema_pb2.LogicalType(urn=PYTHON_ANY_URN)) else: # TODO(bhulette): Add support for logical types that require arguments return schema_pb2.FieldType( logical_type=schema_pb2.LogicalType( urn=logical_type.urn(), representation=typing_to_runner_api( logical_type.representation_type())))
def schema_from_element_type(element_type): # (type) -> schema_pb2.Schema """Get a schema for the given PCollection element_type. Returns schema as a list of (name, python_type) tuples""" if isinstance(element_type, row_type.RowTypeConstraint): # TODO(BEAM-10722): Make sure beam.Row generated schemas are registered and # de-duped return named_fields_to_schema(element_type._fields) elif match_is_named_tuple(element_type): return named_tuple_to_schema(element_type) else: raise TypeError( "Attempted to determine schema for unsupported type '%s'" % element_type)
def schema_from_element_type(element_type: type) -> schema_pb2.Schema: """Get a schema for the given PCollection element_type. Returns schema as a list of (name, python_type) tuples""" if isinstance(element_type, row_type.RowTypeConstraint): # TODO(BEAM-10722): Make sure beam.Row generated schemas are registered and # de-duped return named_fields_to_schema(element_type._fields) elif match_is_named_tuple(element_type): return named_tuple_to_schema(element_type) else: raise TypeError( f"Could not determine schema for type hint {element_type!r}. Did you " "mean to create a schema-aware PCollection? See " "https://s.apache.org/beam-python-schemas")
def register_coder_for_schema(schema: NamedTuple, verbose: bool = False) -> None: """Registers a RowCoder for the given schema if hasn't. Notifies the user of what code has been implicitly executed. """ assert match_is_named_tuple(schema), ( 'Schema %s is not a typing.NamedTuple.' % schema) coder = beam.coders.registry.get_coder(schema) if not isinstance(coder, beam.coders.RowCoder): if verbose: _LOGGER.warning( 'Schema %s has not been registered to use a RowCoder. ' 'Automatically registering it by running: ' 'beam.coders.registry.register_coder(%s, ' 'beam.coders.RowCoder)', schema.__name__, schema.__name__) beam.coders.registry.register_coder(schema, beam.coders.RowCoder)
def beam_sql(self, line: str, cell: Optional[str] = None) -> Optional[PValue]: """The beam_sql line/cell magic that executes a Beam SQL. Args: line: the string on the same line after the beam_sql magic. cell: everything else in the same notebook cell as a string. If None, beam_sql is used as line magic. Otherwise, cell magic. Returns None if running into an error or waiting for user input (running on a selected runner remotely), otherwise a PValue as if a SqlTransform is applied. """ input_str = line if cell: input_str += ' ' + cell parsed = self._parser.parse(input_str.strip().split()) if not parsed: # Failed to parse inputs, let the parser handle the exit. return output_name = parsed.output_name verbose = parsed.verbose query = parsed.query runner = parsed.runner if output_name and not output_name.isidentifier() or keyword.iskeyword( output_name): on_error( 'The output_name "%s" is not a valid identifier. Please supply a ' 'valid identifier that is not a Python keyword.', line) return if not query: on_error('Please supply the SQL query to be executed.') return if runner and runner not in _SUPPORTED_RUNNERS: on_error( 'Runner "%s" is not supported. Supported runners are %s.', runner, _SUPPORTED_RUNNERS) return query = ' '.join(query) found = find_pcolls(query, pcoll_by_name(), verbose=verbose) schemas = set() main_session = importlib.import_module('__main__') for _, pcoll in found.items(): if not match_is_named_tuple(pcoll.element_type): on_error( 'PCollection %s of type %s is not a NamedTuple. See ' 'https://beam.apache.org/documentation/programming-guide/#schemas ' 'for more details.', pcoll, pcoll.element_type) return register_coder_for_schema(pcoll.element_type, verbose=verbose) # Only care about schemas defined by the user in the main module. if hasattr(main_session, pcoll.element_type.__name__): schemas.add(pcoll.element_type) if runner in ('DirectRunner', None): collect_data_for_local_run(query, found) output_name, output, chain = apply_sql(query, output_name, found) chain.current.schemas = schemas cache_output(output_name, output) return output output_name, current_node, chain = apply_sql( query, output_name, found, False) current_node.schemas = schemas # TODO(BEAM-10708): Move the options setup and result handling to a # separate module when more runners are supported. if runner == 'DataflowRunner': _ = chain.to_pipeline() _ = DataflowOptionsForm( output_name, pcoll_by_name()[output_name], verbose).display_for_input() return None else: raise ValueError('Unsupported runner %s.', runner)