Ah the enum I linked in PubsubSchemaIOProvider appears to be dead code, I
don't think it's referenced anywhere. So I think the implementation should
be done, we'd just need to modify PubsubTableProviderIT to exercise the
proto forma.

On Thu, Mar 18, 2021 at 11:46 AM Daniel Collins <dpcoll...@google.com>
wrote:

> > because there are a couple places where Avro, JSON are still hard-coded
> [2,3]
>
> This is not a blocker, its due to the fact that PubsubTableProvider is
> just a wrapper for PubsubSchemaIOProvider. SchemaIOProvider requires you to
> specify all possible options, TableProvider does not.
>
> On Thu, Mar 18, 2021 at 2:44 PM Brian Hulette <bhule...@google.com> wrote:
>
>> Hi Fernando,
>>
>> Daniel Collins actually added the PayloadSerializerProvider concept very
>> recently [1], which is why it looks like Piotr's code doesn't apply
>> anymore. But the good news is I think that PR gets this task pretty close
>> to completion. It doesn't look like the PR *quite* finished adding support
>> for Proto to PubSubTableProvider though, because there are a couple places
>> where Avro, JSON are still hard-coded [2,3]. For this task to be complete
>> we should have tests of protobuf in PubSubTableProviderIT, which is
>> parameterized by payload type [3].
>>
>> Regarding PubSubIO.readProtos: you're right to point out there's some
>> overlap between the various PubSub readers/writers and the TableProviders.
>> Ideally we'd define the logic for reading/writing Beam Rows with each IO in
>> a single place, but right now most of this logic lives in SQL's
>> TableProviders, and in a few places it's duplicated into the IOs, as with
>> readAvrosWithBeamSchema. For this task I think the right thing to do is use
>> PayloadSerializerProvider.
>>
>> [1] https://github.com/apache/beam/pull/13825
>> [2]
>> https://github.com/apache/beam/blob/3fc2ab10d9f5d5c5b65ecf94ce45861857206674/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubSchemaIOProvider.java#L111
>> [3]
>> https://github.com/apache/beam/blob/3fc2ab10d9f5d5c5b65ecf94ce45861857206674/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/pubsub/PubsubTableProviderIT.java#L108
>>
>> On Thu, Mar 18, 2021 at 10:32 AM Fernando Morales Martinez <
>> fernando.mora...@wizeline.com> wrote:
>>
>>> Hi everyone,
>>> I'm working on the WI mentioned in the subject: "Add Proto support to
>>> Pubsub table provider" and I have a few questions . Sorry for the long mail!
>>>
>>>    - The only method in KafkaTableProvider that performs some logic is
>>>    buildBeamSqlTable. However, when taking a look at the tests in
>>>    KafkaTableProviderIT, the only one that is calling that method appears to
>>>    be testFake2. But if I’m not mistaken, that test doesn’t perform any test
>>>    for the proto case. The only one that tests the proto case is testFake
>>>    test, but that is only creating the KafkaTableProvider and that’s it. I
>>>    wanted to base the PubSubTableProvider on the KafkaTableProvider since 
>>> that
>>>    one supports Proto and looks like it accomplishes that support by using
>>>    PayloadSerializer. Is that correct? Should I follow that path?
>>>
>>>
>>>    - I wanted to base the new code on the commits by Piotr, but a lot
>>>    of the code he submitted appears to have been removed, but I can’t grasp
>>>    the reason. There are several commits referencing one another and the 
>>> Proto
>>>    support. Can you shed some light on that?
>>>
>>>
>>>    - Back to the PubsubTableProvider: PubSubIO, which contains
>>>    reader/writer methods for protos (although via ProtoCoder), is being used
>>>    by the PubsubSchemaIOProvider class for other read/write purposes. Why 
>>> are
>>>    the protos reader/writer not used? Because of the ProtoCoder? Should we
>>>    instead implement new readers/writers by formatting the payload directly,
>>>    in a similar fashion to readAvrosWithBeamSchema?
>>>
>>> Thanks a lot for the help!
>>>
>>> - Fernando Morales
>>>
>>>
>>>
>>>
>>>
>>>
>>>
>>>
>>> *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.*
>>
>>

Reply via email to