[ 
https://issues.apache.org/jira/browse/BEAM-12955?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17436085#comment-17436085
 ] 

Brian Hulette edited comment on BEAM-12955 at 10/29/21, 5:07 PM:
-----------------------------------------------------------------

Ah! ok thank you for the clarification about the TFRecord/tf.train.Example 
duality. It looks like Example is a really flexible protobuf (just a list of 
key-value pairs). Based on the 
[docs|https://www.tensorflow.org/api_docs/python/tf/train/Example] it looks 
like it does have structure, but it's defined in some separate configuration 
files:

{quote}This flexible and compact format allows the storage of large amounts of 
typed data, *but requires that the data shape and use be determined by the 
configuration files and parsers that are used to read and write this 
format.*{quote}

I don't think that would be compatible with what we're discussing here. The 
idea with this task is to generate a schema from a proto message that fully 
represents the structure of your data. Something like:

{code}
message Person {
  string name = 1;
  int64 age = 2;
  bool knowsJavascript = 3;
}
{code}

When this is converted to dataframe you'd get one with a name, age, and 
knowsJavascript column with the appropriate types. If we did that for the 
[Example 
proto|https://github.com/tensorflow/tensorflow/blob/919f693420e35d00c8d0a42100837ae3718f7927/tensorflow/core/example/example.proto#L91],
 the result wouldn't really match the structure of your data. It would just 
have a single column "features" with a list encapsulating all the key-value 
pairs.

That being said, I think integrating TF(X) with beam schemas and the dataframe 
API would be really powerful and we should look into it. I filed BEAM-13150 for 
that, let's move any more discussion over there. I am aware of some work from 
the TFX team to use pyarrow with Beam, I can definitely engage them to make 
sure we don't duplicate effort.


was (Author: bhulette):
Ah! ok thank you for the clarification about the TFRecord/tf.train.Example 
duality. It looks like Example is a really flexible protobuf (just a list of 
key-value pairs). Based on the 
[docs|https://www.tensorflow.org/api_docs/python/tf/train/Example] it looks 
like it does have structure, but it's defined in some separate configuration 
files:

{quote}This flexible and compact format allows the storage of large amounts of 
typed data, *but requires that the data shape and use be determined by the 
configuration files and parsers that are used to read and write this 
format.*{quote}

I don't think that would be compatible with what we're discussing here. The 
idea with this task is to generate a schema from a proto message that fully 
represents the structure of your data. Something like:

{code}
message Person {
  string name = 1;
  int64 age = 2;
  bool knowsJavascript = 3;
}
{code}

When this is converted to dataframe you'd get one with a name, age, and 
knowsJavascript column with the appropriate types. If we did that for the 
[Example 
proto](https://github.com/tensorflow/tensorflow/blob/919f693420e35d00c8d0a42100837ae3718f7927/tensorflow/core/example/example.proto#L91),
 the result wouldn't really match the structure of your data. It would just 
have a single column "features" with a list encapsulating all the key-value 
pairs.

That being said, I think integrating TF(X) with beam schemas and the dataframe 
API would be really powerful and we should look into it. I filed BEAM-13150 for 
that, let's move any more discussion over there. I am aware of some work from 
the TFX team to use pyarrow with Beam, I can definitely engage them to make 
sure we don't duplicate effort.

> Add support for inferring Beam Schemas from Python protobuf types
> -----------------------------------------------------------------
>
>                 Key: BEAM-12955
>                 URL: https://issues.apache.org/jira/browse/BEAM-12955
>             Project: Beam
>          Issue Type: Improvement
>          Components: sdk-py-core
>            Reporter: Brian Hulette
>            Assignee: Svetak Vihaan Sundhar
>            Priority: P2
>              Labels: stale-assigned
>
> Just as we can infer a Beam Schema from a NamedTuple type 
> ([code|https://github.com/apache/beam/blob/master/sdks/python/apache_beam/typehints/schemas.py]),
>  we should have support for inferring a schema from a [protobuf-generated 
> Python 
> type|https://developers.google.com/protocol-buffers/docs/pythontutorial].
> This should integrate well with the rest of the schema infrastructure. For 
> example it should be possible to use schema-aware transforms like 
> [SqlTransform|https://beam.apache.org/releases/pydoc/2.32.0/apache_beam.transforms.sql.html#apache_beam.transforms.sql.SqlTransform],
>  
> [Select|https://beam.apache.org/releases/pydoc/2.32.0/apache_beam.transforms.core.html#apache_beam.transforms.core.Select],
>  or 
> [beam.dataframe.convert.to_dataframe|https://beam.apache.org/releases/pydoc/2.32.0/apache_beam.dataframe.convert.html#apache_beam.dataframe.convert.to_dataframe]
>  on a PCollection that is annotated with a protobuf type. For example (using 
> the addressbook_pb2 example from the 
> [tutorial|https://developers.google.com/protocol-buffers/docs/pythontutorial#reading-a-message]):
> {code:python}
> import adressbook_pb2
> import apache_beam as beam
> from apache_beam.dataframe.convert import to_dataframe
> pc = (input_pc | 
> beam.Map(create_person).with_output_type(addressbook_pb2.Person))
> df = to_dataframe(pc) # deferred dataframe with fields id, name, email, ...
> # OR
> pc | beam.transforms.SqlTransform("SELECT name WHERE email = '[email protected]' 
> FROM PCOLLECTION")
> {code}



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to