Hi Mike, Welcome to the project! I'm happy to provide some pointers on this. BEAM-12016 involves adding a couple of new operations to the Beam DataFrame API, so it would be helpful to read up on that to get the necessary background info. Some useful resources include user facing documentation [1,2], and Robert's original design doc [3]. Also the talk that Robert and I gave at Beam Summit 2020 [4,5] introduces the API from the user perspective and provides some implementation details (note we also talk about SqlTransform there which isn't relevant for this jira).
After that all that's left is actually writing the tests and the code! Note that our goal with the Beam DataFrame API is to exactly match what the pandas library does, so most of our tests take the form: - Perform some operation with pandas - Perform the same operation with the Beam DataFrame API - Verify the results are equivalent One place we do this is in pandas_doctests_test.py [6], which follows that pattern with all of the examples from the pandas documentation (see the add_prefix examples here [7]). Currently we've explicitly skipped those examples in pandas_doctests_test.py [8]. So a great first step would be to just remove those two lines and run pandas_doctest_test.py to see how it fails. We also have a lot of tests that follow this pattern in frames_test.py [9]. Those are generally easier to debug, so if it's helpful you may want to add some test-cases for add_suffix and/or add_prefix there. Now for the code. In order to make the tests pass you'll need to add an implementation for add_prefix and add_suffix in frames.py [10]. In the DeferredDataFrame case these two operations will be "elementwise" methods, which we have a special helper for - you can see how it's used for abs here [11]. The DeferredSeries case is more complicated because these operations modify the index of the Series, so we need to note that the operations do *not* preserve any partitioning by Index, reorder_levels is an example of another operation like that [12]. The partitioning concept can be a little tough to wrap your head around at first, I recently wrote up some documentation that could be helpful there [13]. I hope that helps! Please don't hesitate to reach out with more questions, I know there's a lot to take in here. Brian Background: [1] https://beam.apache.org/blog/dataframe-api-preview-available/ [2] https://beam.apache.org/documentation/dsls/dataframes/overview/ [3] https://s.apache.org/beam-dataframes [4] https://s.apache.org/simpler-python-pipelines-2020 [5] https://2020.beamsummit.org/sessions/simpler-python-pipelines/ Code pointers: [6] https://github.com/apache/beam/blob/master/sdks/python/apache_beam/dataframe/pandas_doctests_test.py [7] https://pandas.pydata.org/docs/reference/api/pandas.DataFrame.add_prefix.html [8] https://github.com/apache/beam/blob/437bdeef302cb30abad36db86aef4e89c12eadd4/sdks/python/apache_beam/dataframe/pandas_doctests_test.py#L79-L80 [9] https://github.com/apache/beam/blob/master/sdks/python/apache_beam/dataframe/frames_test.py [10] https://github.com/apache/beam/blob/master/sdks/python/apache_beam/dataframe/frames.py [11] https://github.com/apache/beam/blob/8122b33c164f131a5e9c14e740d0169c594727d3/sdks/python/apache_beam/dataframe/frames.py#L301 [12] https://github.com/apache/beam/blob/8122b33c164f131a5e9c14e740d0169c594727d3/sdks/python/apache_beam/dataframe/frames.py#L513 [13] https://github.com/apache/beam/blob/8122b33c164f131a5e9c14e740d0169c594727d3/sdks/python/apache_beam/dataframe/expressions.py#L156 On Thu, Apr 15, 2021 at 1:44 PM Rogelio Miguel Hernández Sandoval < rogelio.hernan...@wizeline.com> wrote: > Hi team, > I'm new to the project and I wanted to start by taking the task BEAM-12016 > <https://issues.apache.org/jira/browse/BEAM-12016>. Do you have any info > or suggestions for its implementation/analysis? > > Thank you > -- > > Mike > > > > > > > > > *This email and its contents (including any attachments) are being sent > toyou on the condition of confidentiality and may be protected by > legalprivilege. Access to this email by anyone other than the intended > recipientis unauthorized. If you are not the intended recipient, please > immediatelynotify the sender by replying to this message and delete the > materialimmediately from your system. Any further use, dissemination, > distributionor reproduction of this email is strictly prohibited. Further, > norepresentation is made with respect to any content contained in this > email.*