-
Notifications
You must be signed in to change notification settings - Fork 4.3k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Better errors when inputs are omitted. #28289
Changes from 5 commits
604987d
66c5c4a
5b82597
16a8ccb
f0a9c62
889c0cf
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -61,6 +61,16 @@ def provided_transforms(self) -> Iterable[str]: | |
"""Returns a list of transform type names this provider can handle.""" | ||
raise NotImplementedError(type(self)) | ||
|
||
def requires_inputs(self, typ: str, args: Mapping[str, Any]) -> bool: | ||
"""Returns whether this transform requires inputs. | ||
|
||
Specifically, if this returns True and inputs are not provided than an error | ||
will be thrown. | ||
|
||
This is best-effort, primarily for better and earlier error messages. | ||
""" | ||
return not typ.startswith('Read') | ||
|
||
def create_transform( | ||
self, | ||
typ: str, | ||
|
@@ -125,9 +135,7 @@ def __init__(self, urns, service): | |
def provided_transforms(self): | ||
return self._urns.keys() | ||
|
||
def create_transform(self, type, args, yaml_create_transform): | ||
if callable(self._service): | ||
self._service = self._service() | ||
def schema_transforms(self): | ||
if self._schema_transforms is None: | ||
try: | ||
self._schema_transforms = { | ||
|
@@ -138,8 +146,19 @@ def create_transform(self, type, args, yaml_create_transform): | |
except Exception: | ||
# It's possible this service doesn't vend schema transforms. | ||
self._schema_transforms = {} | ||
return self._schema_transforms | ||
|
||
def requires_inputs(self, typ, args): | ||
if self._urns[type] in self.schema_transforms(): | ||
return bool(self.schema_transforms()[self._urns[type]].inputs) | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Unfortunately relying on schema transform's inputs/outputs is not very reliable (and maybe completely unreliable). They do not depend on the configuration passed to the transforms making them basically useless for determining anything other than inputs/outputs that are present in all possible configurations. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Yeah, it'd be nice to be able to query about the requirements of a configured transform before invoking it, but that's a larger change... Most (though not all) transforms have fixed inputs at least, so this'll cover those. |
||
else: | ||
return super().requires_inputs(typ, args) | ||
|
||
def create_transform(self, type, args, yaml_create_transform): | ||
if callable(self._service): | ||
self._service = self._service() | ||
urn = self._urns[type] | ||
if urn in self._schema_transforms: | ||
if urn in self.schema_transforms(): | ||
return external.SchemaAwareExternalTransform( | ||
urn, self._service, rearrange_based_on_discovery=True, **args) | ||
else: | ||
|
@@ -345,8 +364,9 @@ def fn_takes_side_inputs(fn): | |
|
||
|
||
class InlineProvider(Provider): | ||
def __init__(self, transform_factories): | ||
def __init__(self, transform_factories, no_input_transforms=()): | ||
self._transform_factories = transform_factories | ||
self._no_input_transforms = set(no_input_transforms) | ||
|
||
def available(self): | ||
return True | ||
|
@@ -360,6 +380,14 @@ def create_transform(self, type, args, yaml_create_transform): | |
def to_json(self): | ||
return {'type': "InlineProvider"} | ||
|
||
def requires_inputs(self, typ, args): | ||
if typ in self._no_input_transforms: | ||
return False | ||
elif hasattr(self._transform_factories[typ], '_yaml_requires_inputs'): | ||
return self._transform_factories[typ]._yaml_requires_inputs | ||
else: | ||
return super().requires_inputs(typ, args) | ||
|
||
|
||
class MetaInlineProvider(InlineProvider): | ||
def create_transform(self, type, args, yaml_create_transform): | ||
|
@@ -491,30 +519,30 @@ def _parse_window_spec(spec): | |
# TODO: Triggering, etc. | ||
return beam.WindowInto(window_fn) | ||
|
||
return InlineProvider( | ||
dict({ | ||
'Create': create, | ||
'PyMap': lambda fn: beam.Map( | ||
python_callable.PythonCallableWithSource(fn)), | ||
'PyMapTuple': lambda fn: beam.MapTuple( | ||
python_callable.PythonCallableWithSource(fn)), | ||
'PyFlatMap': lambda fn: beam.FlatMap( | ||
python_callable.PythonCallableWithSource(fn)), | ||
'PyFlatMapTuple': lambda fn: beam.FlatMapTuple( | ||
python_callable.PythonCallableWithSource(fn)), | ||
'PyFilter': lambda keep: beam.Filter( | ||
python_callable.PythonCallableWithSource(keep)), | ||
'PyTransform': fully_qualified_named_transform, | ||
'PyToRow': lambda fields: beam.Select( | ||
**{ | ||
name: python_callable.PythonCallableWithSource(fn) | ||
for (name, fn) in fields.items() | ||
}), | ||
'WithSchema': with_schema, | ||
'Flatten': Flatten, | ||
'WindowInto': WindowInto, | ||
'GroupByKey': beam.GroupByKey, | ||
})) | ||
return InlineProvider({ | ||
'Create': create, | ||
'PyMap': lambda fn: beam.Map( | ||
python_callable.PythonCallableWithSource(fn)), | ||
'PyMapTuple': lambda fn: beam.MapTuple( | ||
python_callable.PythonCallableWithSource(fn)), | ||
'PyFlatMap': lambda fn: beam.FlatMap( | ||
python_callable.PythonCallableWithSource(fn)), | ||
'PyFlatMapTuple': lambda fn: beam.FlatMapTuple( | ||
python_callable.PythonCallableWithSource(fn)), | ||
'PyFilter': lambda keep: beam.Filter( | ||
python_callable.PythonCallableWithSource(keep)), | ||
'PyTransform': fully_qualified_named_transform, | ||
'PyToRow': lambda fields: beam.Select( | ||
**{ | ||
name: python_callable.PythonCallableWithSource(fn) | ||
for (name, fn) in fields.items() | ||
}), | ||
'WithSchema': with_schema, | ||
'Flatten': Flatten, | ||
'WindowInto': WindowInto, | ||
'GroupByKey': beam.GroupByKey, | ||
}, | ||
no_input_transforms=('Create', )) | ||
|
||
|
||
class PypiExpansionService: | ||
|
@@ -585,6 +613,9 @@ def available(self) -> bool: | |
def provided_transforms(self) -> Iterable[str]: | ||
return self._transforms.keys() | ||
|
||
def requires_inputs(self, typ, args): | ||
return self._underlying_provider.requires_inputs(typ, args) | ||
|
||
def create_transform( | ||
self, | ||
typ: str, | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Presuming this is to provide a reasonable default? Seems like it might lead to surprising false positives though?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yep, just a reasonable default. Most transforms that do not start with Read require an input by, the main exception being SQL (where we can't tell and the "typical" use of selecting on PCollection would be good to error on). One can explicitly state
inputs: []
to suppress this error.