damccorm commented on code in PR #25919:
URL: https://github.com/apache/beam/pull/25919#discussion_r1144933210


##########
sdks/python/gen_protos.py:
##########
@@ -123,7 +123,19 @@ def generate_urn_files(out_dir, api_path):
   This is executed at build time rather than dynamically on import to ensure
   that it is compatible with static type checkers like mypy.
   """
-  from google._upb import _message
+  try:
+    from google._upb import _message
+    list_types = (
+        list,
+        _message.RepeatedScalarContainer,
+        _message.RepeatedCompositeContainer,
+    )  # pylint: disable=c-extension-no-member
+  except ImportError:
+    from google.protobuf.internal import containers
+    list_types = (
+        list,
+        containers.RepeatedScalarFieldContainer,
+        containers.RepeatedCompositeFieldContainer)

Review Comment:
   Trying to follow why we need this - is this from having an old protobuf 
installed? It looks like we previously used `google.protobuf.pyext._message` 
before 
https://github.com/apache/beam/commit/5cb1711c39c2ebc3005363f938ca4a4eee774333, 
is there a reason we're now using `google.protobuf.internal.containers`?
   
   cc/ @AnandInguva @tvalentyn 



##########
sdks/python/apache_beam/dataframe/frames.py:
##########
@@ -5394,6 +5396,24 @@ def func(df, *args, **kwargs):
         frame_base._elementwise_method(inplace_name, inplace=True,
                                        base=pd.DataFrame))
 
+# Allow dataframe | SchemaTransform
+def _create_maybe_elementwise_or(base):
+  elementwise = frame_base._elementwise_method(
+      '__or__', restrictions={'level': None}, base=base)
+
+  def _maybe_elementwise_or(self, right):
+    if isinstance(right, PTransform):
+      return convert.to_dataframe(convert.to_pcollection(self) | right)

Review Comment:
   I think this is fine, it is worth calling out that we're opening users up to 
doing an inefficient thing where they go:
   
   ```
   df = convert.to_dataframe(pc)
   df2 = df | MyDfOperation()
   df3 = df2 | MySchemaTransform() | MySchemaTransform2() | MySchemaTransform3()
   result = convert.to_dataframe(df3)
   ```
   
   which would be (much?) more efficient written as:
   
   ```
   df = convert.to_dataframe(pc)
   df2 = df | MyDfOperation()
   result = convert.to_dataframe(df2) | MySchemaTransform() | 
MySchemaTransform2() | MySchemaTransform3()
   ```
   
   because this avoids the repeated `to_dataframe`/`to_pcollection` transition. 
The former is probably more natural though, especially if you have real df 
operations mixed in there even if its less efficient. I think the user 
experience still trumps the efficiency loss, but it might be something we want 
to doc.



-- 
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.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to