Skip to content
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

[BEAM-7746] Add python type hints (part 2) #10367

Merged
merged 14 commits into from Jan 14, 2020

Conversation

chadrik
Copy link
Contributor

@chadrik chadrik commented Dec 12, 2019

This is part 2 of #9056.

Unlike part 1 (#9915) this goes beyond simple type comments. It introduces changes that could affect runtime behavior, though I was careful to avoid doing so, unless it's noted as a bug.


Thank you for your contribution! Follow this checklist to help us incorporate your contribution quickly and easily:

  • Choose reviewer(s) and mention them in a comment (R: @username).
  • Format the pull request title like [BEAM-XXX] Fixes bug in ApproximateQuantiles, where you replace BEAM-XXX with the appropriate JIRA issue, if applicable. This will automatically link the pull request to the issue.
  • If this contribution is large, please file an Apache Individual Contributor License Agreement.

See the Contributor Guide for more tips on how to make review process smoother.

Post-Commit Tests Status (on master branch)

Lang SDK Apex Dataflow Flink Gearpump Samza Spark
Go Build Status --- --- Build Status --- --- Build Status
Java Build Status Build Status Build Status Build Status
Build Status
Build Status
Build Status Build Status Build Status
Build Status
Build Status
Python Build Status
Build Status
Build Status
Build Status
--- Build Status
Build Status
Build Status
Build Status
--- --- Build Status
XLang --- --- --- Build Status --- --- ---

Pre-Commit Tests Status (on master branch)

--- Java Python Go Website
Non-portable Build Status Build Status
Build Status
Build Status Build Status
Portable --- Build Status --- ---

See .test-infra/jenkins/README for trigger phrase, status and link of all Jenkins jobs.

@chadrik
Copy link
Contributor Author

chadrik commented Dec 12, 2019

R: @robertwb
R: @udim

Some of these were already reviewed in the overarching PR (#9056), but I did not include them in part 1 because I was trying to focus on type comments in part 1.

I tried to give some context for each change in the commit description. It may be worth considering not squashing history when merging this.

@pabloem
Copy link
Member

pabloem commented Dec 12, 2019

Thanks Chad. I decided to squash the previous PR because I saw some commits containing 'fixup' type comments. Sorry if that was not the intention (guess I should have checked..). Thanks for explicitly requesting the 'merge' option this time : )

@chadrik
Copy link
Contributor Author

chadrik commented Dec 12, 2019 via email

@chadrik
Copy link
Contributor Author

chadrik commented Jan 5, 2020

Hi, now that the holidays are over, I'd like to bump this to the tops of people's review queue!

Copy link
Member

@udim udim left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

partial review, about half way done

sdks/python/apache_beam/coders/coder_impl.py Show resolved Hide resolved
sdks/python/apache_beam/portability/__init__.py Outdated Show resolved Hide resolved
sdks/python/gen_protos.py Outdated Show resolved Hide resolved
sdks/python/gen_protos.py Show resolved Hide resolved
sdks/python/apache_beam/pvalue.py Outdated Show resolved Hide resolved
sdks/python/apache_beam/utils/profiler.py Show resolved Hide resolved
sdks/python/apache_beam/runners/common.py Show resolved Hide resolved
sdks/python/apache_beam/runners/common.py Show resolved Hide resolved
@chadrik
Copy link
Contributor Author

chadrik commented Jan 9, 2020

@udim thanks for the review! very good questions.

@chadrik
Copy link
Contributor Author

chadrik commented Jan 9, 2020

btw, I made some edits to my answers to clarify them, so you should continue the review via github rather than email.

sdks/python/apache_beam/io/vcfio.py Show resolved Hide resolved
@@ -808,7 +810,7 @@ class AppliedPTransform(object):

def __init__(self,
parent,
transform, # type: ptransform.PTransform
transform, # type: Optional[ptransform.PTransform]
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Where is this optional?

Copy link
Contributor Author

@chadrik chadrik Jan 9, 2020

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

in Pipeline.__init__:

    # Stack of transforms generated by nested apply() calls. The stack will
    # contain a root node as an enclosing (parent) node for top transforms.
    self.transforms_stack = [AppliedPTransform(None, None, '', None)]

Best way to deal with this may be a special RootAppliedTransform subclass.


It can also possibly be None in AppliedPTransform.from_runner_api():

    transform = ptransform.PTransform.from_runner_api(proto.spec, context)
    result = AppliedPTransform(
        parent=None,
        transform=transform,
        full_label=proto.unique_name,
        inputs=main_inputs)

This is because PTransform.from_runner_api() returns Optional[PTransform]

  @classmethod
  def from_runner_api(cls,
                      proto,  # type: Optional[beam_runner_api_pb2.FunctionSpec]
                      context  # type: PipelineContext
                     ):
    # type: (...) -> Optional[PTransform]
    if proto is None or not proto.urn:
      return None
    parameter_type, constructor = cls._known_urns[proto.urn]

Copy link
Contributor Author

@chadrik chadrik Jan 9, 2020

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

On the second issue, I should point that mypy knows that proto.spec is not None when we call PTransform. from_runner_api(proto.spec, context) (because proto.spec is always non-None), and we could almost use that knowledge to solve this problem with @overloads of PTransform.from_runner_api(), like this:

  @overload
  @classmethod
  def from_runner_api(cls,
                      proto,  # type: None
                      context  # type: PipelineContext
                     ):
    # type: (...) -> None
    pass

  @overload
  @classmethod
  def from_runner_api(cls,
                      proto,  # type: beam_runner_api_pb2.FunctionSpec
                      context  # type: PipelineContext
                     ):
    # type: (...) -> PTransform
    pass
  
  @classmethod
  def from_runner_api(cls, proto, context):
    if proto is None or not proto.urn:
      return None

Unfortunately, typing can't track whether the value of proto.urn is an empty string, which means that the above overload strategy doesn't actually work. Is there any chance that this could be changed to if proto is None or proto.urn is None?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I don't think it'd ever be None, given that an unset proto field is the default value of that field.

sdks/python/apache_beam/pvalue.py Outdated Show resolved Hide resolved
sdks/python/apache_beam/runners/common.py Show resolved Hide resolved
sdks/python/apache_beam/runners/common.py Show resolved Hide resolved
@@ -437,13 +439,15 @@ def invoke_start_bundle(self):
# type: () -> None
"""Invokes the DoFn.start_bundle() method.
"""
assert self.output_processor is not None
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

These asserts should be fast, but have you verified this doesn't impact performance (given this is called for every element of every transform). Or is there another way to declare this for critical parts of the code. (In particular, isinstance can be slow. It's not common convention to disable asserts in Python.)

Copy link
Contributor Author

@chadrik chadrik Jan 9, 2020

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I have not done performance testing on this. Is there a way that we can invoke the perf suite on Jenkins from github?

The underlying issue in this case is that it's possible to instantiate a DoFnInvoker without an output_processor and that's considered ok (by us) as long as you don't call any of the methods that use the output_processor. If you do, then it would raise an exception. The asserts obviously also raise an exception, but it serves as a way to communicate to mypy that you're aware of it, and it can adjust its type analysis within that scope.

Solutions

Easy: Judiciously add type: ignore comments. I say "judiciously" because ignoring an error does not change the type analysis, so similar errors can pop up nearby in the same scope. In this particular case the methods are brief, so a type: ignore comment would suffice.

Not Easy: Rework the code so there's a subclass of DoFnInvoker that always has a non-optional output_processor, and only this class possesses these "safe if you're careful" methods, which would, under that design, always be safe.

The easy solution is fine for this particular case (we're aware there could be an error, but we accept it), but it's not a general solution to this problem. Choosing the right solution for each case takes some consideration.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think we can get rid of output_processor altogether in this code... (It's a bit ugly ever since it was introduced, this typing confirms that.) May not be the case everywhere though.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Ok, how about I change these to type: ignore and you create a ticket to remove output_processor? I would make the ticket but I don't think I have enough context to explain it properly.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

sgtm

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is done. I ended up leaving one assert in invoke_process because it was conditional, and by using one assert I avoided 3 ignores that cluttered up the code. Let me know what you think.

Copy link
Contributor

@robertwb robertwb left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

OK, I've finished going through all the files. Overall it looks good, just a couple more comments.

@@ -265,7 +265,7 @@ def finish(self):

class _StateBackedIterable(object):
def __init__(self,
state_handler,
state_handler, # type: sdk_worker.CachingStateHandler
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is the 'Caching' part necessary here (even if it always is right now)?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

CachingStateHandler does not inherit from StateHandler nor does it implement its abstract methods

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Oh... :(

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yeah, maybe worth creating another issue for this. Could be a nice entry-level task.

@@ -1130,19 +1132,26 @@ def process(self, windowed_value):

@BeamTransformFactory.register_urn(
DATA_INPUT_URN, beam_fn_api_pb2.RemoteGrpcPort)
def create(factory, transform_id, transform_proto, grpc_port, consumers):
def create_source_runner(
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

These registered constructors (necessarily) all have the same signature. Is there a way to declare that in a common place? (The return type is always Operation, what type is not ever introspected.)

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think the best way to reduce the noise here would be make the registration more object oriented.

Quick sketch:

class OpCreator(Generic[OperatorT]):
  def __init__(
      self,
      factory,  # type: BeamTransformFactory
      transform_id,  # type: str
      transform_proto,  # type: beam_runner_api_pb2.PTransform
      consumers  # type: Dict[str, List[operations.Operation]]
  ):
    self.factory = factory
    self.transform_id = transform_id
    self.transform_proto = transform_proto
    self.consumers = consumers

  def create(self, parameter):
    # type: (Any) -> OperatorT
    raise NotImplementedError

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

That's an idea. It would still add a level of indirection and boilerplate...

@overload
def register_urn(cls,
urn, # type: str
parameter_type, # type: None
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Idea: could we unify these and update all callers that currently pass None to pass bytes?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

that would be nice from a typing simplicity standpoint. not sure about the implications of that though.

Copy link
Contributor

@robertwb robertwb left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

If you can make the changes ignore OutputProcessor and update the code to reflect that pipeline=None is transient, this looks good to go into me. Thanks.

@robertwb
Copy link
Contributor

Run all tests

@robertwb
Copy link
Contributor

Everything looks good except a little bit of lint: https://builds.apache.org/job/beam_PreCommit_PythonLint_Commit/1894/

Minor adjustments to runtime code required to silence certain errors.

Two common patterns:
- explicitly return None from functions that also return non-None
- assert that optional attributes are non-None before using them, if there are no other conditionals present to ensure this.
…uild process

This fixes numerous errors generated throughout the code because mypy cannot
track the dynamic setattr binding that was done by common_urns.

The change also necessitated updating a few doctrings to prevent this error:

more than one target found for cross-reference u'DisplayData':
 apache_beam.transforms.display.DisplayData
 apache_beam.portability.api.beam_runner_api_pb2_urns.DisplayData
There are several places in the code where it is assumed that these are part of the abstract StateSpec.
…w.StateSampler.reset()

statesampler_slow.StateSampler does not have _states_by_name attribute.  Only its fast counterpart does.
This gives us a type that we can use to ensure all handlers meet the same protocol
Note that this means that tests that were previously being masked by other
tests with the same name will now be run.

There is a fix included for one such test.
@chadrik
Copy link
Contributor Author

chadrik commented Jan 14, 2020

Run all tests

@chadrik
Copy link
Contributor Author

chadrik commented Jan 14, 2020

I'm a bit confused: I don't see the Jenkins jobs listed here any more. Why would that be?

@robertwb
Copy link
Contributor

Run all tests

@robertwb
Copy link
Contributor

(As per the discussion on the dev list, Apache Infra made a change to block all tests from being run by non-comitters.)

@chadrik
Copy link
Contributor Author

chadrik commented Jan 14, 2020 via email

@chadrik
Copy link
Contributor Author

chadrik commented Jan 14, 2020

@robertwb @udim I think everything is addressed. I melded the review notes into the commits, so I'll leave it up to you whether you want to squash or not.

@robertwb robertwb merged commit d03404d into apache:master Jan 14, 2020
@chadrik
Copy link
Contributor Author

chadrik commented Jan 14, 2020

@robertwb @udim would you rather I make 6 new PRs for 6 typing changes (each covers a distinct topic), or lump them together into one PR. The lumped PR is smaller than this one.

@robertwb
Copy link
Contributor

If they're orthogonal enough, let's make 6.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

None yet

4 participants