Brian, Coders have a provider model where the provider can be queried to
resolve for a given type and the providers are resolved in a specific
order. This gave the flexibility to handle situations like the one you
described.

On Wed, Aug 19, 2020 at 12:30 AM <robert.butc...@natwestmarkets.com> wrote:

> Hi Brian,
>
>
>
> Many thanks for your mail.
>
>
>
> Yes I figured that one out in the end from the docs, but many thanks for
> confirming.
>
>
>
> I did subsequently discover some other issues with protoBuf-derived
> schemas (essentially they don’t seem to be properly supported by
> BigQueryIO.Write or allow for optional fields) but I posted a separate
> message on the dev channel covering this.
>
>
>
> Kind regards,
>
>
>
> Rob
>
>
>
> *From:* Brian Hulette [mailto:bhule...@google.com]
> *Sent:* 18 August 2020 20:50
> *To:* user
> *Subject:* Re: Registering Protobuf schema
>
>
>
>
> *********************************************
> "This is an external email. Do you know who has sent it? Can you be sure
> that any links and attachments contained within it are safe? If in any
> doubt, use the Phishing Reporter Button in your Outlook client or forward
> the email as an attachment to ~ I've Been Phished"
> *********************************************
>
> Hi Robert,
> Sorry for the late reply on this. I think you should be able to do this by
> registering it in your pipeline's SchemaRegistry manually, like so:
>
>
>
>   Pipeline p;
>
>   p.getSchemaRegistry().registerSchemaProvider(Fx.class,
> ProtoMessageSchema.class);
>
> Of course this isn't quite as nice as just adding the DefualtSchema
> annotation to a class you control. Maybe we should consider some global
> config that would always use schemas for proto-generated classes.
>
>
> Brian
>
>
>
> On Sun, Jul 12, 2020 at 9:56 AM Kaymak, Tobias <tobias.kay...@ricardo.ch>
> wrote:
>
> This sounds like it is related to the problem I'm trying to solve. (In my
> case having a Java POJO containing a protobuf backed-class and trying to
> generate a Beam Schema from it.)
>
> I would be very interested to a solution to this as well :)
>
>
>
> On Tue, Jul 7, 2020 at 2:22 PM <robert.butc...@natwestmarkets.com> wrote:
>
> Hi All,
>
>
>
> I have a BEAM pipeline where I am reading data from some parquet files and
> converting them into a different format based on protobuf generated classes.
>
>
>
> I wish to associate a schema (derived from the protobuf classes) for my
> PCollections.  What is the appropriate way to do this with
> protobuf-generated classes?
>
>
>
> Code excerpt:
>
>
>
> PCollection<Fx> result = input.apply("FXFilePattern", FileIO.*match*
> ().filepattern(fxDataFilePattern))
>         .apply("FXReadMatches", FileIO.*readMatches*())
>         .apply("FXReadParquetFile", ParquetIO.*readFiles*(fxAvroSchema))
>         .apply("MapFXToProto", ParDo.*of*(MapFXToProto.*of*()));
> boolean hasSchema = result.hasSchema();  // returns false
>
>
>
> With thanks in advance.
>
>
>
> Kind regards,
>
>
>
> Rob
>
>
>
> *Robert Butcher*
>
> *Technical Architect | Foundry/SRS | NatWest Markets*
>
> WeWork, 10 Devonshire Square, London, EC2M 4AE
>
> Mobile +44 (0) 7414 730866 <+44%207414%20730866>
>
>
>
> This email is classified as *CONFIDENTIAL* unless otherwise stated.
>
>
>
>
>
> This communication and any attachments are confidential and intended
> solely for the addressee. If you are not the intended recipient please
> advise us immediately and delete it. Unless specifically stated in the
> message or otherwise indicated, you may not duplicate, redistribute or
> forward this message and any attachments are not intended for distribution
> to, or use by any person or entity in any jurisdiction or country where
> such distribution or use would be contrary to local law or regulation.
> NatWest Markets Plc  or any affiliated entity ("NatWest Markets") accepts
> no responsibility for any changes made to this message after it was sent.
>
> Unless otherwise specifically indicated, the contents of this
> communication and its attachments are for information purposes only and
> should not be regarded as an offer or solicitation to buy or sell a product
> or service, confirmation of any transaction, a valuation, indicative price
> or an official statement. Trading desks may have a position or interest
> that is inconsistent with any views expressed in this message. In
> evaluating the information contained in this message, you should know that
> it could have been previously provided to other clients and/or internal
> NatWest Markets personnel, who could have already acted on it.
>
> NatWest Markets cannot provide absolute assurances that all electronic
> communications (sent or received) are secure, error free, not corrupted,
> incomplete or virus free and/or that they will not be lost, mis-delivered,
> destroyed, delayed or intercepted/decrypted by others. Therefore NatWest
> Markets disclaims all liability with regards to electronic communications
> (and the contents therein) if they are corrupted, lost destroyed, delayed,
> incomplete, mis-delivered, intercepted, decrypted or otherwise
> misappropriated by others.
>
> Any electronic communication that is conducted within or through NatWest
> Markets systems will be subject to being archived, monitored and produced
> to regulators and in litigation in accordance with NatWest Markets’ policy
> and local laws, rules and regulations. Unless expressly prohibited by local
> law, electronic communications may be archived in countries other than the
> country in which you are located, and may be treated in accordance with the
> laws and regulations of the country of each individual included in the
> entire chain.
>
> Copyright NatWest Markets Plc. All rights reserved. See
> https://www.nwm.com/disclaimer for further risk disclosure.
>
>
> This communication and any attachments are confidential and intended
> solely for the addressee. If you are not the intended recipient please
> advise us immediately and delete it. Unless specifically stated in the
> message or otherwise indicated, you may not duplicate, redistribute or
> forward this message and any attachments are not intended for distribution
> to, or use by any person or entity in any jurisdiction or country where
> such distribution or use would be contrary to local law or regulation.
> NatWest Markets Plc, NatWest Markets N.V., NatWest Markets Securities Japan
> Limited and/or NatWest Markets Securities Inc. (collectively "NatWest
> Markets") accepts no responsibility for any changes made to this message
> after it was sent.
> This communication, where prepared by the sales and trading desk or desk
> strategists, may be marketing material, desk strategy and/or trader
> commentary. It is not a product of the research department. This material
> may constitute an invitation to consider entering into a derivatives
> transaction under U.S. CFTC Regulations sections 1.71 and 23.605, where
> applicable, but is not a binding offer to buy/sell any financial
> instrument. The views of the author may differ from others at NatWest
> Markets.
> Unless otherwise specifically indicated, the contents of this
> communication and its attachments are for information purposes only and
> should not be regarded as an offer or solicitation to buy or sell a product
> or service, confirmation of any transaction, a valuation, indicative price
> or an official statement. Trading desks may have a position or interest
> that is inconsistent with any views expressed in this message. In
> evaluating the information contained in this message, you should know that
> it could have been previously provided to other clients and/or internal
> NatWest Markets personnel, who could have already acted on it.
> NatWest Markets cannot provide absolute assurances that all electronic
> communications (sent or received) are secure, error free, not corrupted,
> incomplete or virus free and/or that they will not be lost, mis-delivered,
> destroyed, delayed or intercepted/decrypted by others. Therefore NatWest
> Markets disclaims all liability with regards to electronic communications
> (and the contents therein) if they are corrupted, lost destroyed, delayed,
> incomplete, mis-delivered, intercepted, decrypted or otherwise
> misappropriated by others.
> Any electronic communication that is conducted within or through NatWest
> Markets systems will be subject to being archived, monitored and produced
> to regulators and in litigation in accordance with NatWest Markets’ policy
> and local laws, rules and regulations. Unless expressly prohibited by local
> law, electronic communications may be archived in countries other than the
> country in which you are located, and may be treated in accordance with the
> laws and regulations of the country of each individual included in the
> entire chain.
> Copyright © NatWest Markets Plc. All rights reserved. See
> https://www.nwm.com/disclaimer for further risk disclosure (the agency
> arrangements referred to in the further risk disclosure between NatWest
> Markets Plc and NatWest Markets N.V. are not applicable to branches of
> NatWest Markets N.V.).
>

Reply via email to