>
> Is it actually necessary for a PTransform that is configured via the
> Schema mechanism to also be one that uses RowCoder? Those strike me as two
> separate concerns and unnecessarily limiting
>

+1 to this, I have a wip branch somewhere with a coder for
https://github.com/apache/incubator-fury/ to benchmark performance in Beam
as a coder and compare it to the published benchmarks.
With a portable encoding scheme across a broad selection of languages it
could be an interesting candidate for a custom row oriented serialization
scheme tied to schema awareness in Beam.
Decoupling the coder from schemas seems like it would be a prerequisite
change to improve columnar transfer and processing by removing the need to
transpose a batch to/from Row as is currently the case with the Arrow
extension.

On Thu, Mar 7, 2024 at 6:18 PM Ahmed Abualsaud via dev <dev@beam.apache.org>
wrote:

> In the same vein of reducing boilerplate to make a schema-aware transform,
> I've opened a small PR to relax the requirement of 3 methods:
> https://github.com/apache/beam/pull/30560
>
> Would appreciate someone taking a look!
>
> On Thu, Jun 29, 2023 at 9:43 AM Ahmed Abualsaud <ahmedabuals...@google.com>
> wrote:
>
>> Can someone take a quick look at
>> https://github.com/apache/beam/pull/27202? If things look good, let's
>> try getting it in before the release cut as I'm also updating our
>> cross-language documentation and would like to include these changes.
>>
>> Thank you,
>> Ahmed
>>
>> On Thu, Jun 22, 2023 at 8:06 PM Reuven Lax <re...@google.com> wrote:
>>
>>> The goal was to make schema transforms as efficient as hand-written
>>> coders. We found the avro encoding/decoding to often be quite inefficient,
>>> which is one reason we didn't use it for schemas.
>>>
>>> Our schema encoding is internal to Beam though, and not suggested for
>>> use external to a pipeline. For sources or sinks we still recommend using
>>> Avro (or proto).
>>>
>>> On Thu, Jun 22, 2023 at 4:14 PM Robert Bradshaw <rober...@google.com>
>>> wrote:
>>>
>>>> On Thu, Jun 22, 2023 at 2:19 PM Ahmed Abualsaud <
>>>> ahmedabuals...@google.com> wrote:
>>>>
>>>>> Thank you all for your input. I have a PR for the changes I mentioned
>>>>> in my initial email: https://github.com/apache/beam/pull/27202.
>>>>> Please review when you get a chance!
>>>>>
>>>>> > perhaps we should consider just going to something Avro for portable
>>>>> coding rather than something custom
>>>>>
>>>>> Did you mean using some Avro object (GenericRecord?) besides Beam Row
>>>>> elements? We would still run into the problem Cham mentioned earlier (of
>>>>> making sure existing PTransform inputs/outputs are compatible with
>>>>> cross-language-valid types).
>>>>>
>>>>
>>>> I don't remember why Avro was rejected in favor of our own encoding
>>>> format, but it probably doesn't make sense to revisit that without
>>>> understanding the full history. I do know that avro versioning and diamond
>>>> dependencies cause a lot of pain for users and there's a concerted effort
>>>> to remove Avro from Beam core altogether.
>>>>
>>>> In any case, this is quite orthogonal to the proposal here which we
>>>> should move forward on.
>>>>
>>>>
>>>>> On Tue, May 30, 2023 at 10:53 PM Byron Ellis <byronel...@google.com>
>>>>> wrote:
>>>>>
>>>>>> Sure, I get that… though perhaps we should consider just going to
>>>>>> something Avro for portable coding rather than something custom.
>>>>>>
>>>>>> On Tue, May 30, 2023 at 2:22 PM Chamikara Jayalath <
>>>>>> chamik...@google.com> wrote:
>>>>>>
>>>>>>> Input/output PCollection types at least have to be portable Beam
>>>>>>> types [1] for cross-language to work.
>>>>>>>
>>>>>>> I think we restricted schema-aware transforms to PCollection<Row>
>>>>>>> since Row was expected to be an efficient replacement for arbitrary
>>>>>>> portable Beam types (not sure how true that is in practice currently).
>>>>>>>
>>>>>>> Thanks,
>>>>>>> Cham
>>>>>>>
>>>>>>> [1]
>>>>>>> https://github.com/apache/beam/blob/b9730952a7abf60437ee85ba2df6dd30556d6560/model/pipeline/src/main/proto/org/apache/beam/model/pipeline/v1/beam_runner_api.proto#L829
>>>>>>>
>>>>>>> On Tue, May 30, 2023 at 1:54 PM Byron Ellis <byronel...@google.com>
>>>>>>> wrote:
>>>>>>>
>>>>>>>> Is it actually necessary for a PTransform that is configured via
>>>>>>>> the Schema mechanism to also be one that uses RowCoder? Those strike 
>>>>>>>> me as
>>>>>>>> two separate concerns and unnecessarily limiting.
>>>>>>>>
>>>>>>>> On Tue, May 30, 2023 at 1:29 PM Chamikara Jayalath <
>>>>>>>> chamik...@google.com> wrote:
>>>>>>>>
>>>>>>>>> +1 for the simplification.
>>>>>>>>>
>>>>>>>>> On Tue, May 30, 2023 at 12:33 PM Robert Bradshaw <
>>>>>>>>> rober...@google.com> wrote:
>>>>>>>>>
>>>>>>>>>> Yeah. Essentially one needs do (1) name the arguments and (2)
>>>>>>>>>> implement the transform. Hopefully (1) could be done in a concise 
>>>>>>>>>> way that
>>>>>>>>>> allows for easy consumption from both Java and cross-language.
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>> +1 but I think the hard part today is to convert existing
>>>>>>>>> PTransforms to be schema-aware transform compatible (for example, 
>>>>>>>>> change
>>>>>>>>> input/output types and make sure parameters take Beam Schema 
>>>>>>>>> compatible
>>>>>>>>> types). But this makes sense for new transforms.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>> On Tue, May 30, 2023 at 12:25 PM Byron Ellis <
>>>>>>>>>> byronel...@google.com> wrote:
>>>>>>>>>>
>>>>>>>>>>> Or perhaps the other way around? If you have a Schema we can
>>>>>>>>>>> auto-generate the associated builder on the PTransform? Either way, 
>>>>>>>>>>> more
>>>>>>>>>>> DRY.
>>>>>>>>>>>
>>>>>>>>>>> On Tue, May 30, 2023 at 10:59 AM Robert Bradshaw via dev <
>>>>>>>>>>> dev@beam.apache.org> wrote:
>>>>>>>>>>>
>>>>>>>>>>>> +1 to this simplification, it's a historical artifact that
>>>>>>>>>>>> provides no value.
>>>>>>>>>>>>
>>>>>>>>>>>> I would love it if we also looked into ways to auto-generate
>>>>>>>>>>>> the SchemaTransformProvider (e.g. via introspection if a transform 
>>>>>>>>>>>> takes a
>>>>>>>>>>>> small number of arguments, or uses the standard builder 
>>>>>>>>>>>> pattern...),
>>>>>>>>>>>> ideally with something as simple as adding a decorator to the 
>>>>>>>>>>>> PTransform
>>>>>>>>>>>> itself.
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> On Tue, May 30, 2023 at 7:42 AM Ahmed Abualsaud via dev <
>>>>>>>>>>>> dev@beam.apache.org> wrote:
>>>>>>>>>>>>
>>>>>>>>>>>>> Hey everyone,
>>>>>>>>>>>>>
>>>>>>>>>>>>> I was looking at how we use SchemaTransforms in our expansion
>>>>>>>>>>>>> service. From what I see, there may be a redundant step in 
>>>>>>>>>>>>> developing
>>>>>>>>>>>>> SchemaTransforms. Currently, we have 3 pieces:
>>>>>>>>>>>>> - SchemaTransformProvider [1]
>>>>>>>>>>>>> - A configuration object
>>>>>>>>>>>>> - SchemaTransform [2]
>>>>>>>>>>>>>
>>>>>>>>>>>>> The API is generally used like this:
>>>>>>>>>>>>> 1. The SchemaTransformProvider takes a configuration object
>>>>>>>>>>>>> and returns a SchemaTransform
>>>>>>>>>>>>> 2. The SchemaTransform is used to build a PTransform according
>>>>>>>>>>>>> to the configuration
>>>>>>>>>>>>>
>>>>>>>>>>>>> In these steps, the SchemaTransform class seems unnecessary.
>>>>>>>>>>>>> We can combine the two steps if we have SchemaTransformProvider 
>>>>>>>>>>>>> return the
>>>>>>>>>>>>> PTransform directly.
>>>>>>>>>>>>>
>>>>>>>>>>>>> We can then remove the SchemaTransform class as it will be
>>>>>>>>>>>>> obsolete. This should be safe to do; the only place it's used in 
>>>>>>>>>>>>> our API is
>>>>>>>>>>>>> here [3], and that can be simplified if we make this change (we'd 
>>>>>>>>>>>>> just trim
>>>>>>>>>>>>> `.buildTransform()` off the end as `provider.from(configRow)`
>>>>>>>>>>>>> will directly return the PTransform).
>>>>>>>>>>>>>
>>>>>>>>>>>>> I'd like to first mention that I was not involved in
>>>>>>>>>>>>> the design process of this API so I may be missing some 
>>>>>>>>>>>>> information on why
>>>>>>>>>>>>> it was set up this way.
>>>>>>>>>>>>>
>>>>>>>>>>>>> A few developers already raised questions about how there's
>>>>>>>>>>>>> seemingly unnecessary boilerplate involved in making a Java 
>>>>>>>>>>>>> transform
>>>>>>>>>>>>> portable. I wasn't involved in the design process of this API so 
>>>>>>>>>>>>> I may be
>>>>>>>>>>>>> missing some information, but my assumption is this was designed 
>>>>>>>>>>>>> to follow
>>>>>>>>>>>>> the pattern of the previous iteration of this API (SchemaIO):
>>>>>>>>>>>>> SchemaIOProvider[4] -> SchemaIO[5] -> PTransform. However,
>>>>>>>>>>>>> with the newer SchemaTransformProvider API, we dropped a few 
>>>>>>>>>>>>> methods and
>>>>>>>>>>>>> reduced the SchemaTransform class to have a generic 
>>>>>>>>>>>>> buildTransform()
>>>>>>>>>>>>> method. See the example of PubsubReadSchemaTransformProvider [6], 
>>>>>>>>>>>>> where the
>>>>>>>>>>>>> SchemaTransform interface and buildTransform method are
>>>>>>>>>>>>> implemented just to satisfy the requirement that
>>>>>>>>>>>>> SchemaTransformProvider::from return a SchemaTransform.
>>>>>>>>>>>>>
>>>>>>>>>>>>> I'm bringing this up because if we are looking to encourage
>>>>>>>>>>>>> contribution to cross-language use cases, we should make it 
>>>>>>>>>>>>> simpler and
>>>>>>>>>>>>> less convoluted to develop portable transforms.
>>>>>>>>>>>>>
>>>>>>>>>>>>> There are a number of SchemaTransforms already developed, but
>>>>>>>>>>>>> applying these changes to them should be straightforward. If 
>>>>>>>>>>>>> people think
>>>>>>>>>>>>> this is a good idea, I can open a PR and implement them.
>>>>>>>>>>>>>
>>>>>>>>>>>>> Best,
>>>>>>>>>>>>> Ahmed
>>>>>>>>>>>>>
>>>>>>>>>>>>> [1]
>>>>>>>>>>>>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/SchemaTransformProvider.java
>>>>>>>>>>>>> [2]
>>>>>>>>>>>>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/SchemaTransform.java
>>>>>>>>>>>>> [3]
>>>>>>>>>>>>> https://github.com/apache/beam/blob/d7ded3f07064919c202c81a2c786910e20a834f9/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionServiceSchemaTransformProvider.java#L138
>>>>>>>>>>>>> [4]
>>>>>>>>>>>>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/io/SchemaIOProvider.java
>>>>>>>>>>>>> [5]
>>>>>>>>>>>>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/io/SchemaIO.java
>>>>>>>>>>>>> [6]
>>>>>>>>>>>>> https://github.com/apache/beam/blob/ed1a297904d5f5c743a6aca1a7648e3fb8f02e18/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubReadSchemaTransformProvider.java#L133-L137
>>>>>>>>>>>>>
>>>>>>>>>>>>

Reply via email to