All JMS related properties can be null (@Nullable).

Good point about breaking change.

The part on which I'm not big fan in your proposal is this
getDynamic() method. Maybe we can mimic what I did in JdbcIO with a fn
we can inject to define the destination.

But, ok, I would be happy to review a PR (I can comment on the PR though).

Regards
JB

On Thu, Feb 24, 2022 at 5:16 PM BALLADA Vincent
<[email protected]> wrote:
>
> Hi Jean Baptiste 😉
>
>
>
> Thank you for your feedback, it is interesting.
>
>
>
> In your proposal, the write transform would take a PCollection< JmsRecord>.
>
> JmsRecord has a lot of property that are related to the read operation 
> (jmsRedelivered, correlationId, etc…)
>
> As a result I think it won’t be easy to create it with all relevant 
> properties at late stage of the pipeline, before the write operation.
>
> It would introduce also a breaking change as the current implementation takes 
> a Pcollection<String> as input.
>
> Plus the new standard for IO implementation seems to be parameterized 
> read/write (see annotated document).
>
>
>
> Thanks,
>
>
>
> Regards
>
>
>
> Vincent BALLADA
>
>
>
> De : Jean-Baptiste Onofré <[email protected]>
> Date : jeudi, 24 février 2022 à 16:32
> À : [email protected] <[email protected]>
> Objet : Re: [Proposal] => JMSIO dynamic topic publishing
>
> [EXTERNAL EMAIL : Be CAUTIOUS, especially with links and attachments]
>
> Hi Vincent,
>
> It's Jean-Baptiste, not Jean-François, but it works as well ;)
>
> I got your point, however, I think we can achieve the same with
> JmsRecord. You can always use a DoFn at any part of your pipeline
> (after reading from kafka, redis, whatever), where you can create
> JmsRecord. If JmsRecord contains a property with destination, it can
> be populated dynamically JmsDestination.
>
> So basically, it means that we can add JmsDestinationType on the
> JmsRecord and use it in the sink.
>
> Something lie:
>
> pipeline
>  .apply(...) // returns PCollection<JmsRecord> // JmsReccord can be
> created/populated with destination and destination type
> .apply(JmsIO.write().withConnectionFactory(myConnectionFactory))
>
> I think it's less invasive.
>
> It's basically the same that you propose, but reusing JmsRecord and
> avoiding getDynamic() hook.
>
> Again, I'm not against with your proposal, but we can have something
> more concise.
>
> Just my €0.01 ;)
>
> Regards
> JB
>
> On Thu, Feb 24, 2022 at 3:42 PM BALLADA Vincent
> <[email protected]> wrote:
> >
> > Hi Jean François
> >
> >
> >
> > Please to hear of the author of JmsIO 😊.
> >
> > Many thanks for your suggestion.
> >
> > JmsRecord is used at read time, in most use cases we will use a mapper to 
> > provide an object that will be used in several transform in the pipeline.
> >
> > The destination won’t necessary be included in the read JmsRecord, as for 
> > instance in many pipelines we do data enrichment (form redis, database, 
> > etc…) and the dynamic part of the topic would be extracted at any stage of 
> > the pipeline.
> >
> > So for me it would be more flexible to not be tied with JmsRecord.
> >
> >
> >
> > Regards
> >
> >
> >
> > Vincent BALLADA
> >
> >
> >
> > De : Jean-Baptiste Onofré <[email protected]>
> > Date : samedi, 19 février 2022 à 06:40
> > À : [email protected] <[email protected]>
> > Objet : Re: [Proposal] => JMSIO dynamic topic publishing
> >
> > [EXTERNAL EMAIL : Be CAUTIOUS, especially with links and attachments]
> >
> > Hi Vincent,
> >
> > It looks interesting. Another possible approach is to have some
> > implicit (instead of being explicit) but defining the destination on
> > the JmsRecord. If the JmsRecord contains the destination (that could
> > be "dynamic"), we use it, overriding the destination provided on the
> > IO configuration.
> > Thoughts ?
> >
> > Regards
> > JB
> >
> > NB: I'm the original author of JmsIO ;)
> >
> > On Fri, Feb 18, 2022 at 7:00 PM BALLADA Vincent
> > <[email protected]> wrote:
> > >
> > > Hi all
> > >
> > >
> > >
> > > Here is a proposal to implement the ability to publish on dynamic topics 
> > > with JMSIO:
> > >
> > > https://docs.google.com/document/d/1IY4_e5g1g71XvTLL4slHRyVfX7ByiwjD_de3WGsBQXg/edit?usp=sharing
> > >
> > >
> > >
> > > There is also a JIRA issue:
> > >
> > > https://issues.apache.org/jira/browse/BEAM-13608
> > >
> > >
> > >
> > > Best regards
> > >
> > >
> > >
> > > Vincent BALLADA
> > >
> > >
> > > Confidential C
> > >
> > > -- Disclaimer ------------------------------------
> > > Ce message ainsi que les eventuelles pieces jointes constituent une 
> > > correspondance privee et confidentielle a l'attention exclusive du 
> > > destinataire designe ci-dessus. Si vous n'etes pas le destinataire du 
> > > present message ou une personne susceptible de pouvoir le lui delivrer, 
> > > il vous est signifie que toute divulgation, distribution ou copie de 
> > > cette transmission est strictement interdite. Si vous avez recu ce 
> > > message par erreur, nous vous remercions d'en informer l'expediteur par 
> > > telephone ou de lui retourner le present message, puis d'effacer 
> > > immediatement ce message de votre systeme.
> > >
> > > *** This e-mail and any attachments is a confidential correspondence 
> > > intended only for use of the individual or entity named above. If you are 
> > > not the intended recipient or the agent responsible for delivering the 
> > > message to the intended recipient, you are hereby notified that any 
> > > disclosure, distribution or copying of this communication is strictly 
> > > prohibited. If you have received this communication in error, please 
> > > notify the sender by phone or by replying this message, and then delete 
> > > this message from your system.
> >
> > <p></p>
> >
> >
> > Confidential C
> >
> > -- Disclaimer ------------------------------------
> > Ce message ainsi que les eventuelles pieces jointes constituent une 
> > correspondance privee et confidentielle a l'attention exclusive du 
> > destinataire designe ci-dessus. Si vous n'etes pas le destinataire du 
> > present message ou une personne susceptible de pouvoir le lui delivrer, il 
> > vous est signifie que toute divulgation, distribution ou copie de cette 
> > transmission est strictement interdite. Si vous avez recu ce message par 
> > erreur, nous vous remercions d'en informer l'expediteur par telephone ou de 
> > lui retourner le present message, puis d'effacer immediatement ce message 
> > de votre systeme.
> >
> > *** This e-mail and any attachments is a confidential correspondence 
> > intended only for use of the individual or entity named above. If you are 
> > not the intended recipient or the agent responsible for delivering the 
> > message to the intended recipient, you are hereby notified that any 
> > disclosure, distribution or copying of this communication is strictly 
> > prohibited. If you have received this communication in error, please notify 
> > the sender by phone or by replying this message, and then delete this 
> > message from your system.
>
> <p></p>
>
>
> Confidential C
>
> -- Disclaimer ------------------------------------
> Ce message ainsi que les eventuelles pieces jointes constituent une 
> correspondance privee et confidentielle a l'attention exclusive du 
> destinataire designe ci-dessus. Si vous n'etes pas le destinataire du present 
> message ou une personne susceptible de pouvoir le lui delivrer, il vous est 
> signifie que toute divulgation, distribution ou copie de cette transmission 
> est strictement interdite. Si vous avez recu ce message par erreur, nous vous 
> remercions d'en informer l'expediteur par telephone ou de lui retourner le 
> present message, puis d'effacer immediatement ce message de votre systeme.
>
> *** This e-mail and any attachments is a confidential correspondence intended 
> only for use of the individual or entity named above. If you are not the 
> intended recipient or the agent responsible for delivering the message to the 
> intended recipient, you are hereby notified that any disclosure, distribution 
> or copying of this communication is strictly prohibited. If you have received 
> this communication in error, please notify the sender by phone or by replying 
> this message, and then delete this message from your system.

Reply via email to