You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@beam.apache.org by GitBox <gi...@apache.org> on 2020/03/04 02:26:22 UTC

[GitHub] [beam] chadrik opened a new pull request #11038: [BEAM-7746] More typing fixes

chadrik opened a new pull request #11038: [BEAM-7746] More typing fixes
URL: https://github.com/apache/beam/pull/11038
 
 
   Another round of fixes.  
   
   After this and the filesystem PR are merged, there will be only 32 errors remaining.  I'd like to hand the remainder of the errors over to @robertwb and @udim to resolve, since many of the require some specific Beam knowledge to solve.   I will of course be happy to help (we could connect on slack via the Beam channel?).
   
   Then we can put the PythonLint job into effect!
   
   
   ------------------------
   
   Thank you for your contribution! Follow this checklist to help us incorporate your contribution quickly and easily:
   
    - [ ] [**Choose reviewer(s)**](https://beam.apache.org/contribute/#make-your-change) 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.
    - [ ] Update `CHANGES.md` with noteworthy changes.
    - [ ] If this contribution is large, please file an Apache [Individual Contributor License Agreement](https://www.apache.org/licenses/icla.pdf).
   
   See the [Contributor Guide](https://beam.apache.org/contribute) for more tips on [how to make review process smoother](https://beam.apache.org/contribute/#make-reviewers-job-easier).
   
   Post-Commit Tests Status (on master branch)
   ------------------------------------------------------------------------------------------------
   
   Lang | SDK | Apex | Dataflow | Flink | Gearpump | Samza | Spark
   --- | --- | --- | --- | --- | --- | --- | ---
   Go | [![Build Status](https://builds.apache.org/job/beam_PostCommit_Go/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Go/lastCompletedBuild/) | --- | --- | [![Build Status](https://builds.apache.org/job/beam_PostCommit_Go_VR_Flink/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Go_VR_Flink/lastCompletedBuild/) | --- | --- | [![Build Status](https://builds.apache.org/job/beam_PostCommit_Go_VR_Spark/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Go_VR_Spark/lastCompletedBuild/)
   Java | [![Build Status](https://builds.apache.org/job/beam_PostCommit_Java/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java/lastCompletedBuild/) | [![Build Status](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Apex/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Apex/lastCompletedBuild/) | [![Build Status](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Dataflow/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Dataflow/lastCompletedBuild/)<br>[![Build Status](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Dataflow_Java11/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Dataflow_Java11/lastCompletedBuild/) | [![Build Status](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Flink/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Flink/lastCompletedBuild/)<br>[![Build Status](https://builds.apache.org/job/beam_PostCommit_Java_PVR_Flink_Batch/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_PVR_Flink_Batch/lastCompletedBuild/)<br>[![Build Status](https://builds.apache.org/job/beam_PostCommit_Java_PVR_Flink_Streaming/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_PVR_Flink_Streaming/lastCompletedBuild/) | [![Build Status](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Gearpump/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Gearpump/lastCompletedBuild/) | [![Build Status](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Samza/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Samza/lastCompletedBuild/) | [![Build Status](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Spark/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Spark/lastCompletedBuild/)<br>[![Build Status](https://builds.apache.org/job/beam_PostCommit_Java_PVR_Spark_Batch/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_PVR_Spark_Batch/lastCompletedBuild/)<br>[![Build Status](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming/lastCompletedBuild/)
   Python | [![Build Status](https://builds.apache.org/job/beam_PostCommit_Python2/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Python2/lastCompletedBuild/)<br>[![Build Status](https://builds.apache.org/job/beam_PostCommit_Python35/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Python35/lastCompletedBuild/)<br>[![Build Status](https://builds.apache.org/job/beam_PostCommit_Python36/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Python36/lastCompletedBuild/)<br>[![Build Status](https://builds.apache.org/job/beam_PostCommit_Python37/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Python37/lastCompletedBuild/) | --- | [![Build Status](https://builds.apache.org/job/beam_PostCommit_Py_VR_Dataflow/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Py_VR_Dataflow/lastCompletedBuild/)<br>[![Build Status](https://builds.apache.org/job/beam_PostCommit_Py_VR_Dataflow_V2/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Py_VR_Dataflow_V2/lastCompletedBuild/)<br>[![Build Status](https://builds.apache.org/job/beam_PostCommit_Py_ValCont/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Py_ValCont/lastCompletedBuild/) | [![Build Status](https://builds.apache.org/job/beam_PreCommit_Python2_PVR_Flink_Cron/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PreCommit_Python2_PVR_Flink_Cron/lastCompletedBuild/)<br>[![Build Status](https://builds.apache.org/job/beam_PostCommit_Python35_VR_Flink/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Python35_VR_Flink/lastCompletedBuild/) | --- | --- | [![Build Status](https://builds.apache.org/job/beam_PostCommit_Python_VR_Spark/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Python_VR_Spark/lastCompletedBuild/)
   XLang | --- | --- | --- | [![Build Status](https://builds.apache.org/job/beam_PostCommit_XVR_Flink/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_XVR_Flink/lastCompletedBuild/) | --- | --- | [![Build Status](https://builds.apache.org/job/beam_PostCommit_XVR_Spark/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_XVR_Spark/lastCompletedBuild/)
   
   Pre-Commit Tests Status (on master branch)
   ------------------------------------------------------------------------------------------------
   
   --- |Java | Python | Go | Website
   --- | --- | --- | --- | ---
   Non-portable | [![Build Status](https://builds.apache.org/job/beam_PreCommit_Java_Cron/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PreCommit_Java_Cron/lastCompletedBuild/) | [![Build Status](https://builds.apache.org/job/beam_PreCommit_Python_Cron/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PreCommit_Python_Cron/lastCompletedBuild/)<br>[![Build Status](https://builds.apache.org/job/beam_PreCommit_PythonLint_Cron/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PreCommit_PythonLint_Cron/lastCompletedBuild/) | [![Build Status](https://builds.apache.org/job/beam_PreCommit_Go_Cron/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PreCommit_Go_Cron/lastCompletedBuild/) | [![Build Status](https://builds.apache.org/job/beam_PreCommit_Website_Cron/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PreCommit_Website_Cron/lastCompletedBuild/) 
   Portable | --- | [![Build Status](https://builds.apache.org/job/beam_PreCommit_Portable_Python_Cron/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PreCommit_Portable_Python_Cron/lastCompletedBuild/) | --- | ---
   
   See [.test-infra/jenkins/README](https://github.com/apache/beam/blob/master/.test-infra/jenkins/README.md) for trigger phrase, status and link of all Jenkins jobs.
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] udim commented on a change in pull request #11038: [BEAM-7746] More typing fixes

Posted by GitBox <gi...@apache.org>.
udim commented on a change in pull request #11038: [BEAM-7746] More typing fixes
URL: https://github.com/apache/beam/pull/11038#discussion_r393289270
 
 

 ##########
 File path: sdks/python/apache_beam/metrics/metric.py
 ##########
 @@ -56,7 +55,7 @@ class Metrics(object):
   @staticmethod
   def get_namespace(namespace):
     # type: (Union[Type, str]) -> str
-    if inspect.isclass(namespace):
+    if isinstance(namespace, type):
 
 Review comment:
   Hopefully no one uses old-style classes any more (types.ClassType).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] udim commented on a change in pull request #11038: [BEAM-7746] More typing fixes

Posted by GitBox <gi...@apache.org>.
udim commented on a change in pull request #11038: [BEAM-7746] More typing fixes
URL: https://github.com/apache/beam/pull/11038#discussion_r394034914
 
 

 ##########
 File path: sdks/python/apache_beam/ml/gcp/naturallanguageml.py
 ##########
 @@ -74,7 +75,9 @@ def __init__(
   def to_dict(document):
     # type: (Document) -> Mapping[str, Optional[str]]
     if document.from_gcs:
-      dict_repr = {'gcs_content_uri': document.content}
+      dict_repr = {
+          'gcs_content_uri': document.content
+      }  # type: Dict[str, Optional[str]]
 
 Review comment:
   `document.content` is not Optional

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] udim commented on a change in pull request #11038: [BEAM-7746] More typing fixes

Posted by GitBox <gi...@apache.org>.
udim commented on a change in pull request #11038: [BEAM-7746] More typing fixes
URL: https://github.com/apache/beam/pull/11038#discussion_r394021216
 
 

 ##########
 File path: sdks/python/apache_beam/io/range_trackers.py
 ##########
 @@ -42,7 +48,7 @@
 _LOGGER = logging.getLogger(__name__)
 
 
-class OffsetRangeTracker(iobase.RangeTracker):
+class OffsetRangeTracker(iobase.RangeTracker[int]):
   """A 'RangeTracker' for non-negative positions of type 'long'."""
 
 Review comment:
   s/long/int/

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] chadrik commented on issue #11038: [BEAM-7746] More typing fixes

Posted by GitBox <gi...@apache.org>.
chadrik commented on issue #11038: [BEAM-7746] More typing fixes
URL: https://github.com/apache/beam/pull/11038#issuecomment-597396356
 
 
   Tests passing!  
   
   Have a look when you get a chance.   I'm hoping this one is pretty straight-forward.
   
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] chadrik commented on a change in pull request #11038: [BEAM-7746] More typing fixes

Posted by GitBox <gi...@apache.org>.
chadrik commented on a change in pull request #11038: [BEAM-7746] More typing fixes
URL: https://github.com/apache/beam/pull/11038#discussion_r396690657
 
 

 ##########
 File path: sdks/python/apache_beam/transforms/core.py
 ##########
 @@ -1300,12 +1300,13 @@ def to_runner_api_parameter(self, context):
           common_urns.requirements.REQUIRES_STATEFUL_PROCESSING.urn)
     from apache_beam.runners.common import DoFnSignature
     sig = DoFnSignature(self.fn)
-    is_splittable = sig.is_splittable_dofn()
 
 Review comment:
   > Not sure if checking get_restriction_coder() return type instead of is_splittable_dofn() is future proof.
   
   `get_restriction_coder()` calls `is_splittable_dofn()` and returns `None` if it's not splittable.  So I interpreted a `None` result from this method to mean "is not splittable". 
   
   ```python
     def get_restriction_coder(self):
       # type: () -> Optional[TupleCoder]
   
       """Get coder for a restriction when processing an SDF. """
       if self.is_splittable_dofn():
         return TupleCoder([
             (self.get_restriction_provider().restriction_coder()),
             (self.get_watermark_estimator_provider().estimator_state_coder())
         ])
       else:
         return None
   ```
   
   > I don't understand the change, from a mypy correctness perspective.
   
   Here's the problem:
   
   ```python
       if is_splittable:
         restriction_coder = sig.get_restriction_coder()  #  returns Optional[TupleCoder]
         restriction_coder_id = context.coders.get_id(restriction_coder)  # does not accept Optional!
       else:	    else:
         restriction_coder_id = None
   ```
   
   With my changes, we naturally drop the optionality before passing the value to `context.coders.get_id()`.  We also avoid a redundant call to `is_splittable_dofn()`, FWIW.
   
   I see two options:
   
   1) keep my changes and update the documentation of `get_restriction_coder()` to clarify that `None` result indicates "is not splittable"
   2) revert my changes and add `assert restriction_coder is None` before the call to `context.coders.get_id()`
   
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] udim commented on a change in pull request #11038: [BEAM-7746] More typing fixes

Posted by GitBox <gi...@apache.org>.
udim commented on a change in pull request #11038: [BEAM-7746] More typing fixes
URL: https://github.com/apache/beam/pull/11038#discussion_r394029843
 
 

 ##########
 File path: sdks/python/apache_beam/io/iobase.py
 ##########
 @@ -244,7 +272,7 @@ def is_bounded(self):
     return True
 
 
-class RangeTracker(object):
+class RangeTracker(Generic[PositionT]):
 
 Review comment:
   I'm not sure. Positions might be opaque byte strings, where splitting happens externally. This might be important for cross-language transforms.
   
   @lukecwik, @robertwb, @chamikaramj , do you have opinion on RangeTracker position types? Should they all support the Position protocol (defined above)?
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] chadrik commented on issue #11038: [BEAM-7746] More typing fixes

Posted by GitBox <gi...@apache.org>.
chadrik commented on issue #11038: [BEAM-7746] More typing fixes
URL: https://github.com/apache/beam/pull/11038#issuecomment-597961227
 
 
   Run Python PreCommit

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] udim commented on a change in pull request #11038: [BEAM-7746] More typing fixes

Posted by GitBox <gi...@apache.org>.
udim commented on a change in pull request #11038: [BEAM-7746] More typing fixes
URL: https://github.com/apache/beam/pull/11038#discussion_r394026348
 
 

 ##########
 File path: sdks/python/apache_beam/io/range_trackers.py
 ##########
 @@ -233,19 +254,25 @@ class OrderedPositionRangeTracker(iobase.RangeTracker):
   UNSTARTED = object()
 
   def __init__(self, start_position=None, stop_position=None):
+    # type: (Optional[iobase.PositionT], Optional[iobase.PositionT]) -> None
     self._start_position = start_position
     self._stop_position = stop_position
     self._lock = threading.Lock()
-    self._last_claim = self.UNSTARTED
+    # the return on investment for properly typing this is low. cast it.
 
 Review comment:
   Did you mean that the ROI is high, not low?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] udim commented on issue #11038: [BEAM-7746] More typing fixes

Posted by GitBox <gi...@apache.org>.
udim commented on issue #11038: [BEAM-7746] More typing fixes
URL: https://github.com/apache/beam/pull/11038#issuecomment-599654214
 
 
   Looking at this today

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] chadrik commented on a change in pull request #11038: [BEAM-7746] More typing fixes

Posted by GitBox <gi...@apache.org>.
chadrik commented on a change in pull request #11038: [BEAM-7746] More typing fixes
URL: https://github.com/apache/beam/pull/11038#discussion_r393187810
 
 

 ##########
 File path: sdks/python/apache_beam/runners/worker/sdk_worker.py
 ##########
 @@ -724,8 +738,9 @@ def pull_responses():
     reader.start()
 
   def done(self):
+    # type: () -> None
     self._done = True
-    self._requests.put(self._DONE)
+    self._requests.put(self._DONE)  # type: ignore[arg-type]
 
 Review comment:
   another opportunity for the `Sentinel` pattern

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] udim commented on a change in pull request #11038: [BEAM-7746] More typing fixes

Posted by GitBox <gi...@apache.org>.
udim commented on a change in pull request #11038: [BEAM-7746] More typing fixes
URL: https://github.com/apache/beam/pull/11038#discussion_r405835011
 
 

 ##########
 File path: sdks/python/apache_beam/coders/coders.py
 ##########
 @@ -387,8 +387,11 @@ def register_structured_urn(urn, cls):
     """Register a coder that's completely defined by its urn and its
     component(s), if any, which are passed to construct the instance.
     """
-    cls.to_runner_api_parameter = (
-        lambda self, unused_context: (urn, None, self._get_component_coders()))
+    setattr(
 
 Review comment:
   I guess set/getattr is cleaner. If this becomes commonplace in the codebase then perhaps we don't need the comment (and mypy will give an error if you attempt to use regular assignment).
   
   ref: https://github.com/python/mypy/issues/2427#issuecomment-259677994

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] chadrik commented on issue #11038: [BEAM-7746] More typing fixes

Posted by GitBox <gi...@apache.org>.
chadrik commented on issue #11038: [BEAM-7746] More typing fixes
URL: https://github.com/apache/beam/pull/11038#issuecomment-596031989
 
 
   R: @udim 
   R: @robertwb 
   
   I broke this down into commits to give some context to the more noteworthy changes. 
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] udim commented on a change in pull request #11038: [BEAM-7746] More typing fixes

Posted by GitBox <gi...@apache.org>.
udim commented on a change in pull request #11038: [BEAM-7746] More typing fixes
URL: https://github.com/apache/beam/pull/11038#discussion_r393383746
 
 

 ##########
 File path: sdks/python/apache_beam/io/iobase.py
 ##########
 @@ -79,6 +82,23 @@
 
 _LOGGER = logging.getLogger(__name__)
 
+
+class Position(Protocol):
+  def __lt__(self, other):
+    pass
+
+  def __le__(self, other):
+    pass
+
+  def __gt__(self, other):
+    pass
+
+  def __ge__(self, other):
+    pass
+
+
+PositionT = TypeVar('PositionT', bound='Position')
 
 Review comment:
   I don't understand the usage of PositionT. Isn't Position already a type?
   It seems that you could replace all uses of PositionT with Position and it'd work the same.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] chadrik commented on issue #11038: [BEAM-7746] More typing fixes

Posted by GitBox <gi...@apache.org>.
chadrik commented on issue #11038: [BEAM-7746] More typing fixes
URL: https://github.com/apache/beam/pull/11038#issuecomment-597230084
 
 
   Run Python PreCommit

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] chadrik commented on a change in pull request #11038: [BEAM-7746] More typing fixes

Posted by GitBox <gi...@apache.org>.
chadrik commented on a change in pull request #11038: [BEAM-7746] More typing fixes
URL: https://github.com/apache/beam/pull/11038#discussion_r396683616
 
 

 ##########
 File path: sdks/python/apache_beam/runners/worker/operations.py
 ##########
 @@ -569,7 +580,7 @@ def __init__(self,
     self.tagged_receivers = None  # type: Optional[_TaggedReceivers]
     # A mapping of timer tags to the input "PCollections" they come in on.
     self.timer_inputs = timer_inputs or {}
-    self.input_info = None  # type: Optional[Tuple[str, str, coders.WindowedValueCoder, MutableMapping[str, str]]]
 
 Review comment:
   correct, I discovered it was only iterating over keys (using `for x in y`), so it only needed to be `Iterable[str]`. 
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] chadrik commented on issue #11038: [BEAM-7746] More typing fixes

Posted by GitBox <gi...@apache.org>.
chadrik commented on issue #11038: [BEAM-7746] More typing fixes
URL: https://github.com/apache/beam/pull/11038#issuecomment-596021712
 
 
   Run Python PreCommit

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] udim commented on a change in pull request #11038: [BEAM-7746] More typing fixes

Posted by GitBox <gi...@apache.org>.
udim commented on a change in pull request #11038: [BEAM-7746] More typing fixes
URL: https://github.com/apache/beam/pull/11038#discussion_r393281202
 
 

 ##########
 File path: sdks/python/apache_beam/runners/worker/operations.py
 ##########
 @@ -569,7 +580,7 @@ def __init__(self,
     self.tagged_receivers = None  # type: Optional[_TaggedReceivers]
     # A mapping of timer tags to the input "PCollections" they come in on.
     self.timer_inputs = timer_inputs or {}
-    self.input_info = None  # type: Optional[Tuple[str, str, coders.WindowedValueCoder, MutableMapping[str, str]]]
 
 Review comment:
   So the MutableMapping hint here was a mistake?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] chadrik commented on a change in pull request #11038: [BEAM-7746] More typing fixes

Posted by GitBox <gi...@apache.org>.
chadrik commented on a change in pull request #11038: [BEAM-7746] More typing fixes
URL: https://github.com/apache/beam/pull/11038#discussion_r396700454
 
 

 ##########
 File path: sdks/python/apache_beam/coders/coders.py
 ##########
 @@ -387,8 +387,11 @@ def register_structured_urn(urn, cls):
     """Register a coder that's completely defined by its urn and its
     component(s), if any, which are passed to construct the instance.
     """
-    cls.to_runner_api_parameter = (
-        lambda self, unused_context: (urn, None, self._get_component_coders()))
+    setattr(
 
 Review comment:
   mypy raises an error when overwriting a method: `Cannot assign to a method`.   using `setattr` avoids the error.  A similar situation arises when setting or getting dynamic attributes.  
   
   We need to decide on a policy for these cases:
   
   1) avoid the error using `setattr` and `getattr`
   2) silence the error with a `# type: ignore[xxx]` comment
   
   The second option provides more context, but could hinge on how you feel about the aesthetics of `type ignore` comments.
   
   Either way, I agree a comment should be made on the preceding line to clarify.
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] chadrik commented on issue #11038: [BEAM-7746] More typing fixes

Posted by GitBox <gi...@apache.org>.
chadrik commented on issue #11038: [BEAM-7746] More typing fixes
URL: https://github.com/apache/beam/pull/11038#issuecomment-597933410
 
 
   Run Python PreCommit

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] udim commented on a change in pull request #11038: [BEAM-7746] More typing fixes

Posted by GitBox <gi...@apache.org>.
udim commented on a change in pull request #11038: [BEAM-7746] More typing fixes
URL: https://github.com/apache/beam/pull/11038#discussion_r394072405
 
 

 ##########
 File path: sdks/python/apache_beam/transforms/core.py
 ##########
 @@ -1300,12 +1300,13 @@ def to_runner_api_parameter(self, context):
           common_urns.requirements.REQUIRES_STATEFUL_PROCESSING.urn)
     from apache_beam.runners.common import DoFnSignature
     sig = DoFnSignature(self.fn)
-    is_splittable = sig.is_splittable_dofn()
 
 Review comment:
   Not sure if checking get_restriction_coder() return type instead of is_splittable_dofn() is future proof.
   
   Also, I don't understand the change, from a mypy correctness perspective.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] chadrik commented on issue #11038: [BEAM-7746] More typing fixes

Posted by GitBox <gi...@apache.org>.
chadrik commented on issue #11038: [BEAM-7746] More typing fixes
URL: https://github.com/apache/beam/pull/11038#issuecomment-598347972
 
 
   Ok, this is stabilized again.  Would love a review. 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] udim commented on a change in pull request #11038: [BEAM-7746] More typing fixes

Posted by GitBox <gi...@apache.org>.
udim commented on a change in pull request #11038: [BEAM-7746] More typing fixes
URL: https://github.com/apache/beam/pull/11038#discussion_r394034514
 
 

 ##########
 File path: sdks/python/apache_beam/io/restriction_trackers.py
 ##########
 @@ -52,6 +55,7 @@ def __hash__(self):
     return hash((type(self), self.start, self.stop))
 
   def split(self, desired_num_offsets_per_split, min_num_offsets_per_split=1):
+    # type: (...) -> Iterator[OffsetRange]
 
 Review comment:
   Input looks like `(int, int)`. Any reason to leave it empty?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] chadrik commented on issue #11038: [BEAM-7746] More typing fixes

Posted by GitBox <gi...@apache.org>.
chadrik commented on issue #11038: [BEAM-7746] More typing fixes
URL: https://github.com/apache/beam/pull/11038#issuecomment-600400560
 
 
   Great comments and questions. I’m in the middle of rolling out our COVID
   plan at work so it may take me a bit to get you proper answers but I’ll
   start chipping away at it as soon as I can.
   
   
   
   On Tue, Mar 17, 2020 at 7:30 PM Udi Meiri <no...@github.com> wrote:
   
   > *@udim* commented on this pull request.
   >
   > This review is taking me forever. I'm at 18 out of 52 files reviewed, but
   > releasing 16 comments I've written so far.
   > ------------------------------
   >
   > In sdks/python/apache_beam/runners/worker/operations.py
   > <https://github.com/apache/beam/pull/11038#discussion_r393281202>:
   >
   > > @@ -569,7 +580,7 @@ def __init__(self,
   >      self.tagged_receivers = None  # type: Optional[_TaggedReceivers]
   >      # A mapping of timer tags to the input "PCollections" they come in on.
   >      self.timer_inputs = timer_inputs or {}
   > -    self.input_info = None  # type: Optional[Tuple[str, str, coders.WindowedValueCoder, MutableMapping[str, str]]]
   >
   > So the MutableMapping hint here was a mistake?
   > ------------------------------
   >
   > In sdks/python/apache_beam/metrics/metric.py
   > <https://github.com/apache/beam/pull/11038#discussion_r393289270>:
   >
   > > @@ -56,7 +55,7 @@ class Metrics(object):
   >    @staticmethod
   >    def get_namespace(namespace):
   >      # type: (Union[Type, str]) -> str
   > -    if inspect.isclass(namespace):
   > +    if isinstance(namespace, type):
   >
   > Hopefully no one uses old-style classes any more (types.ClassType).
   > ------------------------------
   >
   > In sdks/python/mypy.ini
   > <https://github.com/apache/beam/pull/11038#discussion_r393304392>:
   >
   > >  color_output = true
   > -# uncomment this to see how close we are to being complete
   > +# required setting for dmypy:
   > +follow_imports = error
   >
   > Does this mean having to supply all imported modules on the mypy command
   > line?
   > ------------------------------
   >
   > In sdks/python/apache_beam/io/iobase.py
   > <https://github.com/apache/beam/pull/11038#discussion_r393383746>:
   >
   > > +
   > +class Position(Protocol):
   > +  def __lt__(self, other):
   > +    pass
   > +
   > +  def __le__(self, other):
   > +    pass
   > +
   > +  def __gt__(self, other):
   > +    pass
   > +
   > +  def __ge__(self, other):
   > +    pass
   > +
   > +
   > +PositionT = TypeVar('PositionT', bound='Position')
   >
   > I don't understand the usage of PositionT. Isn't Position already a type?
   > It seems that you could replace all uses of PositionT with Position and
   > it'd work the same.
   > ------------------------------
   >
   > In sdks/python/apache_beam/io/iobase.py
   > <https://github.com/apache/beam/pull/11038#discussion_r394017739>:
   >
   > > @@ -95,8 +115,11 @@
   >  #
   >  # Type for start and stop positions are specific to the bounded source and must
   >  # be consistent throughout.
   > -SourceBundle = namedtuple(
   > -    'SourceBundle', 'weight source start_position stop_position')
   > +SourceBundle = NamedTuple(
   > +    'SourceBundle',
   > +    [('weight', Optional[float]), ('source', 'BoundedSource'),
   >
   > It seems that weight is non-Optional.
   > ------------------------------
   >
   > In sdks/python/apache_beam/io/range_trackers.py
   > <https://github.com/apache/beam/pull/11038#discussion_r394021216>:
   >
   > > @@ -42,7 +48,7 @@
   >  _LOGGER = logging.getLogger(__name__)
   >
   >
   > -class OffsetRangeTracker(iobase.RangeTracker):
   > +class OffsetRangeTracker(iobase.RangeTracker[int]):
   >    """A 'RangeTracker' for non-negative positions of type 'long'."""
   >
   > s/long/int/
   > ------------------------------
   >
   > In sdks/python/apache_beam/io/range_trackers.py
   > <https://github.com/apache/beam/pull/11038#discussion_r394026348>:
   >
   > >      self._start_position = start_position
   >      self._stop_position = stop_position
   >      self._lock = threading.Lock()
   > -    self._last_claim = self.UNSTARTED
   > +    # the return on investment for properly typing this is low. cast it.
   >
   > Did you mean that the ROI is high, not low?
   > ------------------------------
   >
   > In sdks/python/apache_beam/io/range_trackers.py
   > <https://github.com/apache/beam/pull/11038#discussion_r394027007>:
   >
   > >    def fraction_to_position(self, fraction, start, end):
   > +    # type: (float, Optional[iobase.PositionT], Optional[iobase.PositionT]) -> Optional[iobase.PositionT]
   >
   > The return value seems to be non-Optional.
   > ------------------------------
   >
   > In sdks/python/apache_beam/io/iobase.py
   > <https://github.com/apache/beam/pull/11038#discussion_r394029843>:
   >
   > > @@ -244,7 +272,7 @@ def is_bounded(self):
   >      return True
   >
   >
   > -class RangeTracker(object):
   > +class RangeTracker(Generic[PositionT]):
   >
   > I'm not sure. Positions might be opaque byte strings, where splitting
   > happens externally. This might be important for cross-language transforms.
   >
   > @lukecwik <https://github.com/lukecwik>, @robertwb
   > <https://github.com/robertwb>, @chamikaramj
   > <https://github.com/chamikaramj> , do you have opinion on RangeTracker
   > position types? Should they all support the Position protocol (defined
   > above)?
   > ------------------------------
   >
   > In sdks/python/apache_beam/io/restriction_trackers.py
   > <https://github.com/apache/beam/pull/11038#discussion_r394034514>:
   >
   > > @@ -52,6 +55,7 @@ def __hash__(self):
   >      return hash((type(self), self.start, self.stop))
   >
   >    def split(self, desired_num_offsets_per_split, min_num_offsets_per_split=1):
   > +    # type: (...) -> Iterator[OffsetRange]
   >
   > Input looks like (int, int). Any reason to leave it empty?
   > ------------------------------
   >
   > In sdks/python/apache_beam/ml/gcp/naturallanguageml.py
   > <https://github.com/apache/beam/pull/11038#discussion_r394034914>:
   >
   > > @@ -74,7 +75,9 @@ def __init__(
   >    def to_dict(document):
   >      # type: (Document) -> Mapping[str, Optional[str]]
   >      if document.from_gcs:
   > -      dict_repr = {'gcs_content_uri': document.content}
   > +      dict_repr = {
   > +          'gcs_content_uri': document.content
   > +      }  # type: Dict[str, Optional[str]]
   >
   > document.content is not Optional
   > ------------------------------
   >
   > In sdks/python/apache_beam/coders/coders.py
   > <https://github.com/apache/beam/pull/11038#discussion_r394038483>:
   >
   > > @@ -387,8 +387,11 @@ def register_structured_urn(urn, cls):
   >      """Register a coder that's completely defined by its urn and its
   >      component(s), if any, which are passed to construct the instance.
   >      """
   > -    cls.to_runner_api_parameter = (
   > -        lambda self, unused_context: (urn, None, self._get_component_coders()))
   > +    setattr(
   >
   > Could you explain (in a comment perhaps) why using setattr here is
   > necessary?
   > ------------------------------
   >
   > In sdks/python/apache_beam/utils/sentinel.py
   > <https://github.com/apache/beam/pull/11038#discussion_r394056361>:
   >
   > > +class Sentinel(enum.Enum):
   > +  """
   > +  A type-safe sentinel class
   > +  """
   > +
   > +  sentinel = object()
   >
   > SG.
   > Sentinel is the type.
   > SPLIT_POINTS_UNKNOWN is the unique value.
   > Inheriting from Enum (vs calling Enum()) simplifies pickling (not sure
   > necessary, but doesn't hurt).
   > ------------------------------
   >
   > In sdks/python/apache_beam/runners/worker/sdk_worker.py
   > <https://github.com/apache/beam/pull/11038#discussion_r394057261>:
   >
   > > @@ -171,7 +172,7 @@ def get_responses():
   >      self._worker_thread_pool.shutdown()
   >      # get_responses may be blocked on responses.get(), but we need to return
   >      # control to its caller.
   > -    self._responses.put(no_more_work)
   > +    self._responses.put(no_more_work)  # type: ignore[arg-type]
   >
   > Sounds like a good idea. There are ~20 places in the code that assign
   > object() as value.
   > One of them is even called READER_THREAD_IS_DONE_SENTINEL. :)
   > ------------------------------
   >
   > In sdks/python/apache_beam/runners/worker/sdk_worker.py
   > <https://github.com/apache/beam/pull/11038#discussion_r394060396>:
   >
   > > @@ -353,6 +359,8 @@ def release(self, instruction_id):
   >      self.cached_bundle_processors[descriptor_id].append(processor)
   >
   >    def shutdown(self):
   > +    # type: (...) -> None
   >
   > This can be () -> None right?
   > Same for the next 2 hints below.
   > ------------------------------
   >
   > In sdks/python/apache_beam/transforms/core.py
   > <https://github.com/apache/beam/pull/11038#discussion_r394072405>:
   >
   > > @@ -1300,12 +1300,13 @@ def to_runner_api_parameter(self, context):
   >            common_urns.requirements.REQUIRES_STATEFUL_PROCESSING.urn)
   >      from apache_beam.runners.common import DoFnSignature
   >      sig = DoFnSignature(self.fn)
   > -    is_splittable = sig.is_splittable_dofn()
   >
   > Not sure if checking get_restriction_coder() return type instead of
   > is_splittable_dofn() is future proof.
   >
   > Also, I don't understand the change, from a mypy correctness perspective.
   >
   > —
   > You are receiving this because you authored the thread.
   > Reply to this email directly, view it on GitHub
   > <https://github.com/apache/beam/pull/11038#pullrequestreview-375545905>,
   > or unsubscribe
   > <https://github.com/notifications/unsubscribe-auth/AAAPOEY43E2LYCWV3TS7KD3RIAW23ANCNFSM4LAZDSSQ>
   > .
   >
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] udim commented on a change in pull request #11038: [BEAM-7746] More typing fixes

Posted by GitBox <gi...@apache.org>.
udim commented on a change in pull request #11038: [BEAM-7746] More typing fixes
URL: https://github.com/apache/beam/pull/11038#discussion_r394017739
 
 

 ##########
 File path: sdks/python/apache_beam/io/iobase.py
 ##########
 @@ -95,8 +115,11 @@
 #
 # Type for start and stop positions are specific to the bounded source and must
 # be consistent throughout.
-SourceBundle = namedtuple(
-    'SourceBundle', 'weight source start_position stop_position')
+SourceBundle = NamedTuple(
+    'SourceBundle',
+    [('weight', Optional[float]), ('source', 'BoundedSource'),
 
 Review comment:
   It seems that `weight` is non-Optional.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] udim commented on a change in pull request #11038: [BEAM-7746] More typing fixes

Posted by GitBox <gi...@apache.org>.
udim commented on a change in pull request #11038: [BEAM-7746] More typing fixes
URL: https://github.com/apache/beam/pull/11038#discussion_r394057261
 
 

 ##########
 File path: sdks/python/apache_beam/runners/worker/sdk_worker.py
 ##########
 @@ -171,7 +172,7 @@ def get_responses():
     self._worker_thread_pool.shutdown()
     # get_responses may be blocked on responses.get(), but we need to return
     # control to its caller.
-    self._responses.put(no_more_work)
+    self._responses.put(no_more_work)  # type: ignore[arg-type]
 
 Review comment:
   Sounds like a good idea. There are ~20 places in the code that assign `object()` as  value.
   One of them is even called `READER_THREAD_IS_DONE_SENTINEL`. :)

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] lukecwik commented on a change in pull request #11038: [BEAM-7746] More typing fixes

Posted by GitBox <gi...@apache.org>.
lukecwik commented on a change in pull request #11038: [BEAM-7746] More typing fixes
URL: https://github.com/apache/beam/pull/11038#discussion_r405860027
 
 

 ##########
 File path: sdks/python/apache_beam/transforms/core.py
 ##########
 @@ -1300,12 +1300,13 @@ def to_runner_api_parameter(self, context):
           common_urns.requirements.REQUIRES_STATEFUL_PROCESSING.urn)
     from apache_beam.runners.common import DoFnSignature
     sig = DoFnSignature(self.fn)
-    is_splittable = sig.is_splittable_dofn()
 
 Review comment:
   It is an error to say is_splittable_dofn is True without returning a restriction coder as well and vice versa.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] udim commented on a change in pull request #11038: [BEAM-7746] More typing fixes

Posted by GitBox <gi...@apache.org>.
udim commented on a change in pull request #11038: [BEAM-7746] More typing fixes
URL: https://github.com/apache/beam/pull/11038#discussion_r394038483
 
 

 ##########
 File path: sdks/python/apache_beam/coders/coders.py
 ##########
 @@ -387,8 +387,11 @@ def register_structured_urn(urn, cls):
     """Register a coder that's completely defined by its urn and its
     component(s), if any, which are passed to construct the instance.
     """
-    cls.to_runner_api_parameter = (
-        lambda self, unused_context: (urn, None, self._get_component_coders()))
+    setattr(
 
 Review comment:
   Could you explain (in a comment perhaps) why using setattr here  is necessary? 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] udim commented on a change in pull request #11038: [BEAM-7746] More typing fixes

Posted by GitBox <gi...@apache.org>.
udim commented on a change in pull request #11038: [BEAM-7746] More typing fixes
URL: https://github.com/apache/beam/pull/11038#discussion_r394056361
 
 

 ##########
 File path: sdks/python/apache_beam/utils/sentinel.py
 ##########
 @@ -0,0 +1,30 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+# pytype: skip-file
+
+from __future__ import absolute_import
+
+import enum
+
+
+class Sentinel(enum.Enum):
+  """
+  A type-safe sentinel class
+  """
+
+  sentinel = object()
 
 Review comment:
   SG. 
   Sentinel is the type.
   SPLIT_POINTS_UNKNOWN is the unique value.
   Inheriting from Enum (vs calling Enum()) simplifies pickling (not sure necessary, but doesn't hurt).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] chadrik commented on a change in pull request #11038: [BEAM-7746] More typing fixes

Posted by GitBox <gi...@apache.org>.
chadrik commented on a change in pull request #11038: [BEAM-7746] More typing fixes
URL: https://github.com/apache/beam/pull/11038#discussion_r396693751
 
 

 ##########
 File path: sdks/python/mypy.ini
 ##########
 @@ -17,16 +17,21 @@
 
 [mypy]
 python_version = 3.6
+files = apache_beam
 ignore_missing_imports = true
-follow_imports = true
-warn_no_return = true
 no_implicit_optional = true
+# warnings:
+warn_no_return = true
 warn_redundant_casts = true
 warn_unused_ignores = true
+# formatting:
 show_error_codes = true
-files = apache_beam
 color_output = true
-# uncomment this to see how close we are to being complete
+# required setting for dmypy:
+follow_imports = error
 
 Review comment:
   mypy picks up the site-packages directory of the python that it's installed into.  Only PEP 561 compliant packages will inform the type analysis (i.e. the package must have a `py.typed` file).  Alternately, you can specify the interpreter for finding site-packages with `--python-executable`:
   
   ```
     --python-executable EXECUTABLE
                               Python executable used for finding PEP 561
                               compliant installed packages and stubs
   ```
   
   This config change did not generate any additional errors for me.  

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] chadrik commented on issue #11038: [BEAM-7746] More typing fixes

Posted by GitBox <gi...@apache.org>.
chadrik commented on issue #11038: [BEAM-7746] More typing fixes
URL: https://github.com/apache/beam/pull/11038#issuecomment-597961202
 
 
   Arg.  Github showed this as failed even after I refreshed, and then as soon as I did `Run Python PreCommit`, it updated to show that it had succeeded briefly and then started the job again, which now failed.  Every time I run it, it seems to fail with a different error.  I don't think they are related to the changes here.
   
   
   
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] chadrik commented on issue #11038: [BEAM-7746] More typing fixes

Posted by GitBox <gi...@apache.org>.
chadrik commented on issue #11038: [BEAM-7746] More typing fixes
URL: https://github.com/apache/beam/pull/11038#issuecomment-597187586
 
 
   Run Python PreCommit

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] udim commented on a change in pull request #11038: [BEAM-7746] More typing fixes

Posted by GitBox <gi...@apache.org>.
udim commented on a change in pull request #11038: [BEAM-7746] More typing fixes
URL: https://github.com/apache/beam/pull/11038#discussion_r394060396
 
 

 ##########
 File path: sdks/python/apache_beam/runners/worker/sdk_worker.py
 ##########
 @@ -353,6 +359,8 @@ def release(self, instruction_id):
     self.cached_bundle_processors[descriptor_id].append(processor)
 
   def shutdown(self):
+    # type: (...) -> None
 
 Review comment:
   This can be `() -> None` right?
   Same for the next 2 hints below.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] chadrik commented on issue #11038: [BEAM-7746] More typing fixes

Posted by GitBox <gi...@apache.org>.
chadrik commented on issue #11038: [BEAM-7746] More typing fixes
URL: https://github.com/apache/beam/pull/11038#issuecomment-598014481
 
 
   Run Python PreCommit

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] chadrik commented on a change in pull request #11038: [BEAM-7746] More typing fixes

Posted by GitBox <gi...@apache.org>.
chadrik commented on a change in pull request #11038: [BEAM-7746] More typing fixes
URL: https://github.com/apache/beam/pull/11038#discussion_r393190055
 
 

 ##########
 File path: sdks/python/apache_beam/utils/sentinel.py
 ##########
 @@ -0,0 +1,30 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+# pytype: skip-file
+
+from __future__ import absolute_import
+
+import enum
+
+
+class Sentinel(enum.Enum):
+  """
+  A type-safe sentinel class
+  """
+
+  sentinel = object()
 
 Review comment:
   If you're interested in how I settled on this design, I documented my experience creating a type-safe sentinel pattern on stackoverflow, here: https://stackoverflow.com/questions/57959664/handling-conditional-logic-sentinel-value-with-mypy/60605919#60605919

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] chadrik commented on issue #11038: [BEAM-7746] More typing fixes

Posted by GitBox <gi...@apache.org>.
chadrik commented on issue #11038: [BEAM-7746] More typing fixes
URL: https://github.com/apache/beam/pull/11038#issuecomment-597339985
 
 
   Run Python PreCommit

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] udim commented on a change in pull request #11038: [BEAM-7746] More typing fixes

Posted by GitBox <gi...@apache.org>.
udim commented on a change in pull request #11038: [BEAM-7746] More typing fixes
URL: https://github.com/apache/beam/pull/11038#discussion_r394027007
 
 

 ##########
 File path: sdks/python/apache_beam/io/range_trackers.py
 ##########
 @@ -283,20 +313,28 @@ def try_split(self, position):
         return None
 
   def fraction_consumed(self):
+    # type: () -> float
     if self._last_claim is self.UNSTARTED:
       return 0
     else:
       return self.position_to_fraction(
           self._last_claim, self._start_position, self._stop_position)
 
+  @classmethod
+  def position_to_fraction(cls, key, start=None, end=None):
+    # type: (iobase.PositionT, Optional[iobase.PositionT], Optional[iobase.PositionT]) -> float
+    raise NotImplementedError
+
   def fraction_to_position(self, fraction, start, end):
+    # type: (float, Optional[iobase.PositionT], Optional[iobase.PositionT]) -> Optional[iobase.PositionT]
 
 Review comment:
   The return value seems to be non-Optional.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] chadrik commented on issue #11038: [BEAM-7746] More typing fixes

Posted by GitBox <gi...@apache.org>.
chadrik commented on issue #11038: [BEAM-7746] More typing fixes
URL: https://github.com/apache/beam/pull/11038#issuecomment-597279936
 
 
   Run Python PreCommit

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] chadrik commented on a change in pull request #11038: [BEAM-7746] More typing fixes

Posted by GitBox <gi...@apache.org>.
chadrik commented on a change in pull request #11038: [BEAM-7746] More typing fixes
URL: https://github.com/apache/beam/pull/11038#discussion_r393187482
 
 

 ##########
 File path: sdks/python/apache_beam/runners/worker/sdk_worker.py
 ##########
 @@ -171,7 +172,7 @@ def get_responses():
     self._worker_thread_pool.shutdown()
     # get_responses may be blocked on responses.get(), but we need to return
     # control to its caller.
-    self._responses.put(no_more_work)
+    self._responses.put(no_more_work)  # type: ignore[arg-type]
 
 Review comment:
   I should probably replace this with the new `Sentinel` pattern. 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] chadrik commented on a change in pull request #11038: [BEAM-7746] More typing fixes

Posted by GitBox <gi...@apache.org>.
chadrik commented on a change in pull request #11038: [BEAM-7746] More typing fixes
URL: https://github.com/apache/beam/pull/11038#discussion_r393185957
 
 

 ##########
 File path: sdks/python/apache_beam/runners/pipeline_context.py
 ##########
 @@ -48,19 +55,33 @@
   from apache_beam.coders.coder_impl import IterableStateReader
   from apache_beam.coders.coder_impl import IterableStateWriter
 
+PortableObjectT = TypeVar('PortableObjectT', bound='PortableObject')
+
+
+class PortableObject(Protocol):
+  def to_runner_api(self, __context):
+    # type: (PipelineContext) -> Any
+    pass
 
-class _PipelineContextMap(object):
+  @classmethod
+  def from_runner_api(cls, __proto, __context):
+    # type: (Any, PipelineContext) -> Any
+    pass
 
 Review comment:
   shall we go with an adjective here instead of a noun:  `Portable`?  

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] udim commented on a change in pull request #11038: [BEAM-7746] More typing fixes

Posted by GitBox <gi...@apache.org>.
udim commented on a change in pull request #11038: [BEAM-7746] More typing fixes
URL: https://github.com/apache/beam/pull/11038#discussion_r405852662
 
 

 ##########
 File path: sdks/python/apache_beam/transforms/core.py
 ##########
 @@ -1300,12 +1300,13 @@ def to_runner_api_parameter(self, context):
           common_urns.requirements.REQUIRES_STATEFUL_PROCESSING.urn)
     from apache_beam.runners.common import DoFnSignature
     sig = DoFnSignature(self.fn)
-    is_splittable = sig.is_splittable_dofn()
 
 Review comment:
   I don't know why, but `ParDoPayload` below accepts both splittable and restriction_coder_id keywords (theoretically splittable might be True while restriction_coder_id is None), so I think it's safer to do this:
   ```py
   is_splittable = sig.is_splittable_dofn()
   restriction_coder = sig.get_restriction_coder()
   if restriction_coder:
     restriction_coder_id = context.coders.get_id(
         restriction_coder)  # type: typing.Optional[str]
   else:
     restriction_coder_id = None
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] udim commented on a change in pull request #11038: [BEAM-7746] More typing fixes

Posted by GitBox <gi...@apache.org>.
udim commented on a change in pull request #11038: [BEAM-7746] More typing fixes
URL: https://github.com/apache/beam/pull/11038#discussion_r405835011
 
 

 ##########
 File path: sdks/python/apache_beam/coders/coders.py
 ##########
 @@ -387,8 +387,11 @@ def register_structured_urn(urn, cls):
     """Register a coder that's completely defined by its urn and its
     component(s), if any, which are passed to construct the instance.
     """
-    cls.to_runner_api_parameter = (
-        lambda self, unused_context: (urn, None, self._get_component_coders()))
+    setattr(
 
 Review comment:
   I guess set/getattr is cleaner. If this becomes commonplace in the codebase then perhaps we don't need the comment (and mypy will give an error if you attempt to use regular assignment).
   
   ref: https://github.com/python/mypy/issues/2427#issuecomment-259677994 (though I don't fully get the examples)

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] chadrik commented on a change in pull request #11038: [BEAM-7746] More typing fixes

Posted by GitBox <gi...@apache.org>.
chadrik commented on a change in pull request #11038: [BEAM-7746] More typing fixes
URL: https://github.com/apache/beam/pull/11038#discussion_r393183860
 
 

 ##########
 File path: sdks/python/apache_beam/io/iobase.py
 ##########
 @@ -244,7 +272,7 @@ def is_bounded(self):
     return True
 
 
-class RangeTracker(object):
+class RangeTracker(Generic[PositionT]):
 
 Review comment:
   @udim For the base `RangeTracker` if we only consider the methods implemented by the class the "position" type could be `Any`, but based on the docs it seemed that any meaningful position would be comparable/sortable.  Let me know if you think we should move the `Position` restriction down to a subclass.  
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] chadrik commented on issue #11038: [BEAM-7746] More typing fixes

Posted by GitBox <gi...@apache.org>.
chadrik commented on issue #11038: [BEAM-7746] More typing fixes
URL: https://github.com/apache/beam/pull/11038#issuecomment-595493314
 
 
   Run Python PreCommit

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] chadrik commented on issue #11038: [BEAM-7746] More typing fixes

Posted by GitBox <gi...@apache.org>.
chadrik commented on issue #11038: [BEAM-7746] More typing fixes
URL: https://github.com/apache/beam/pull/11038#issuecomment-595594678
 
 
   Run Portable_Python PreCommit

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] chadrik commented on a change in pull request #11038: [BEAM-7746] More typing fixes

Posted by GitBox <gi...@apache.org>.
chadrik commented on a change in pull request #11038: [BEAM-7746] More typing fixes
URL: https://github.com/apache/beam/pull/11038#discussion_r396684644
 
 

 ##########
 File path: sdks/python/apache_beam/runners/worker/sdk_worker.py
 ##########
 @@ -353,6 +359,8 @@ def release(self, instruction_id):
     self.cached_bundle_processors[descriptor_id].append(processor)
 
   def shutdown(self):
+    # type: (...) -> None
 
 Review comment:
   correct. looks like I got a little sloppy.   will fix in my next push. 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] udim commented on a change in pull request #11038: [BEAM-7746] More typing fixes

Posted by GitBox <gi...@apache.org>.
udim commented on a change in pull request #11038: [BEAM-7746] More typing fixes
URL: https://github.com/apache/beam/pull/11038#discussion_r393304392
 
 

 ##########
 File path: sdks/python/mypy.ini
 ##########
 @@ -17,16 +17,21 @@
 
 [mypy]
 python_version = 3.6
+files = apache_beam
 ignore_missing_imports = true
-follow_imports = true
-warn_no_return = true
 no_implicit_optional = true
+# warnings:
+warn_no_return = true
 warn_redundant_casts = true
 warn_unused_ignores = true
+# formatting:
 show_error_codes = true
-files = apache_beam
 color_output = true
-# uncomment this to see how close we are to being complete
+# required setting for dmypy:
+follow_imports = error
 
 Review comment:
   Does this mean having to supply all imported modules on the mypy command line?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] chadrik commented on a change in pull request #11038: [BEAM-7746] More typing fixes

Posted by GitBox <gi...@apache.org>.
chadrik commented on a change in pull request #11038: [BEAM-7746] More typing fixes
URL: https://github.com/apache/beam/pull/11038#discussion_r396690657
 
 

 ##########
 File path: sdks/python/apache_beam/transforms/core.py
 ##########
 @@ -1300,12 +1300,13 @@ def to_runner_api_parameter(self, context):
           common_urns.requirements.REQUIRES_STATEFUL_PROCESSING.urn)
     from apache_beam.runners.common import DoFnSignature
     sig = DoFnSignature(self.fn)
-    is_splittable = sig.is_splittable_dofn()
 
 Review comment:
   > Not sure if checking get_restriction_coder() return type instead of is_splittable_dofn() is future proof.
   
   `get_restriction_coder()` calls `is_splittable_dofn()` and returns `None` if it's not splittable.  So I interpreted a `None` result from this method to mean "is not splittable". 
   
   ```python
     def get_restriction_coder(self):
       # type: () -> Optional[TupleCoder]
   
       """Get coder for a restriction when processing an SDF. """
       if self.is_splittable_dofn():
         return TupleCoder([
             (self.get_restriction_provider().restriction_coder()),
             (self.get_watermark_estimator_provider().estimator_state_coder())
         ])
       else:
         return None
   ```
   
   > I don't understand the change, from a mypy correctness perspective.
   
   Here's the problem:
   
   ```python
       if is_splittable:
         restriction_coder = sig.get_restriction_coder()  #  returns Optional[TupleCoder]
         restriction_coder_id = context.coders.get_id(restriction_coder)  # does not accept Optional!
       else:
         restriction_coder_id = None
   ```
   
   With my changes, we naturally drop the optionality before passing the value to `context.coders.get_id()`.  We also avoid a redundant call to `is_splittable_dofn()`, FWIW.
   
   I see two options:
   
   1) keep my changes and update the documentation of `get_restriction_coder()` to clarify that `None` result indicates "is not splittable"
   2) revert my changes and add `assert restriction_coder is None` before the call to `context.coders.get_id()`
   
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services