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
Add LogElements as a Beam PTransform #23879
Conversation
R: @tvalentyn |
Stopping reviewer notifications for this pull request: review requested by someone other than the bot, ceding control |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks a lot!
"event", | ||
datetime(2022, 10, 2, 0, 0, 0, 0, | ||
tzinfo=pytz.UTC).timestamp()), | ||
]) \ |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
nit: it's generally discouraged to use line continuation symbol. It should be not necessary here given you have parentheses, which ensure statement continuation.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Oops, those were leftovers from testing. Removed.
@@ -1105,6 +1105,47 @@ def Iterables(delimiter=None): | |||
Kvs = Iterables | |||
|
|||
|
|||
class LogElements(PTransform): |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
please add this class also to __all__
(line 93). this should also make imports simpler thanks to
from apache_beam.transforms.util import * |
beam/sdks/python/apache_beam/__init__.py
Line 98 in feaa1a2
from apache_beam.transforms import * |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Done.
@@ -49,7 +49,6 @@ COPY --from=build /go/bin/server_python_backend /opt/playground/backend/ | |||
COPY --from=build /go/src/playground/backend/configs /opt/playground/backend/configs/ | |||
|
|||
# Install Python Katas Utils | |||
COPY katas/log_elements.py /go/src/katas/ |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
A lot of playground examples have:
from log_elements import LogElements
. These statements would stop working, so we should either look how to replace them or for a drop-in replacement keep katas/log_elements.py which will simply import LogElements from util.py
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I apologize in advance if my understanding is wrong, but wouldn't the changes in the kata files (such as this one) correct this problem, as the playground backend would pull the updated Python SDK with the changes there after the merge?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
ah, I think you are right, I didn't realize playground just re-used code from katas.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
would using beam.LogElements work instead? that sounds like it would make katas code more uniformly looking
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The beam.LogElements import does work on the tests I ran, and it's definitely a cleaner way to import the transform. I made a new commit with this update.
@@ -1105,6 +1105,47 @@ def Iterables(delimiter=None): | |||
Kvs = Iterables | |||
|
|||
|
|||
class LogElements(PTransform): |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
let's also add typehints like in Reshuffle, see: line 772-773
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Typehints added!
Re Lint errors, you cand find them in Details -> ConsoleOutput, e.g. https://ci-beam.apache.org/job/beam_PreCommit_PythonLint_Commit/16316/consoleText See also: s.apache.org/beam-python-dev-wiki that mentions lint/autoformatting tools |
Codecov Report
@@ Coverage Diff @@
## master #23879 +/- ##
=======================================
Coverage 73.85% 73.85%
=======================================
Files 707 707
Lines 95746 95771 +25
=======================================
+ Hits 70710 70732 +22
- Misses 23720 23723 +3
Partials 1316 1316
Flags with carried forward coverage won't be shown. Click here to find out more.
📣 We’re building smart automated test selection to slash your CI/CD build times. Learn more |
@tvalentyn Thank you for the quick response! All points have been fully addressed except for this one, where I kindly asked you to clarify a piece of information before proceeding. |
Thanks, this looks good to me. Test failures are unrelated. |
cc: @damccorm fyi. |
Closes #23016
Adds LogElements as a valid PTransform with unit tests and updates the previous references to the standalone transform in the katas section.
Thank you for your contribution! Follow this checklist to help us incorporate your contribution quickly and easily:
R: @username
).addresses #123
), if applicable. This will automatically add a link to the pull request in the issue. If you would like the issue to automatically close on merging the pull request, commentfixes #<ISSUE NUMBER>
instead.CHANGES.md
with noteworthy changes.See the Contributor Guide for more tips on how to make review process smoother.
To check the build health, please visit https://github.com/apache/beam/blob/master/.test-infra/BUILD_STATUS.md
GitHub Actions Tests Status (on master branch)
See CI.md for more information about GitHub Actions CI.