[ 
https://issues.apache.org/jira/browse/BEAM-5878?focusedWorklogId=245398&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-245398
 ]

ASF GitHub Bot logged work on BEAM-5878:
----------------------------------------

                Author: ASF GitHub Bot
            Created on: 20/May/19 18:06
            Start Date: 20/May/19 18:06
    Worklog Time Spent: 10m 
      Work Description: tvalentyn commented on pull request #8505: [BEAM-5878] 
add tests for kwonly args in python 3
URL: https://github.com/apache/beam/pull/8505#discussion_r285695694
 
 

 ##########
 File path: 
sdks/python/apache_beam/transforms/transforms_keyword_only_args_py3_test.py
 ##########
 @@ -0,0 +1,111 @@
+#
+# 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.
+#
+
+"""Unit tests for side inputs."""
+
+from __future__ import absolute_import
+
+import logging
+import sys
+import unittest
+
+import apache_beam as beam
+from apache_beam.testing.test_pipeline import TestPipeline
+from apache_beam.testing.util import assert_that
+from apache_beam.testing.util import equal_to
+
+
[email protected](sys.version_info[0] == 2,
+                 'Keyword-Only Arguments are not supported in python 2')
[email protected]('TODO BEAM-5878: support kwonly args in python 3')
+class KeywordOnlyArgsTests(unittest.TestCase):
+  # Enable nose tests running in parallel
+  _multiprocess_can_split_ = True
+
+  def test_side_input_keyword_only_args(self):
+    pipeline = TestPipeline()
+
+    def sort_with_side_inputs(x, *s, reverse=False):
+      for y in s:
+        yield sorted([x] + y, reverse=reverse)
+
+    pcol = pipeline | 'start' >> beam.Create([1, 2])
+    side = pipeline | 'side' >> beam.Create([3, 4])  # 2 values in side input.
+    result1 = pcol | 'compute1' >> beam.FlatMap(
+        sort_with_side_inputs,
+        beam.pvalue.AsList(side), reverse=True)
+    assert_that(result1, equal_to([[4,3,1], [4,3,2]]), label='assert1')
+
+    result2 = pcol | 'compute2' >> beam.FlatMap(
+      sort_with_side_inputs,
+      beam.pvalue.AsIter(side))
+    assert_that(result2, equal_to([[1,3,4], [2,3,4]]), label='assert2')
+
+    result3 = pcol | 'compute3' >> beam.FlatMap(
+      sort_with_side_inputs)
+    assert_that(result3, equal_to([]), label='assert3')
+
+    result4 = pcol | 'compute4' >> beam.FlatMap(
+      sort_with_side_inputs, reverse=True)
+    assert_that(result4, equal_to([]), label='assert4')
+
+    pipeline.run()
+
+  def test_combine_keyword_only_args(self):
+    pipeline = TestPipeline()
+
+    # Keyword-only arguments are not available on Python 2
+    # pylint: disable=syntax-error
 
 Review comment:
   Leftover comment?
 
----------------------------------------------------------------
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:
[email protected]


Issue Time Tracking
-------------------

    Worklog Id:     (was: 245398)
    Time Spent: 3h 50m  (was: 3h 40m)

> Support DoFns with Keyword-only arguments in Python 3.
> ------------------------------------------------------
>
>                 Key: BEAM-5878
>                 URL: https://issues.apache.org/jira/browse/BEAM-5878
>             Project: Beam
>          Issue Type: Sub-task
>          Components: sdk-py-core
>            Reporter: Valentyn Tymofieiev
>            Assignee: Juta Staes
>            Priority: Minor
>          Time Spent: 3h 50m
>  Remaining Estimate: 0h
>
> Python 3.0 [adds a possibility|https://www.python.org/dev/peps/pep-3102/] to 
> define functions with keyword-only arguments. 
> Currently Beam does not handle them correctly. [~ruoyu] pointed out [one 
> place|https://github.com/apache/beam/blob/a56ce43109c97c739fa08adca45528c41e3c925c/sdks/python/apache_beam/typehints/decorators.py#L118]
>  in our codebase that we should fix: in Python in 3.0 inspect.getargspec() 
> will fail on functions with keyword-only arguments, but a new method 
> [inspect.getfullargspec()|https://docs.python.org/3/library/inspect.html#inspect.getfullargspec]
>  supports them.
> There may be implications for our (best-effort) type-hints machinery.
> We should also add a Py3-only unit tests that covers DoFn's with keyword-only 
> arguments once Beam Python 3 tests are in a good shape.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to