Two comments: (1) This approach is basically "within" the processor, as it does use ProcessorContext -- "outside" would mean to use IQ API.
(2) As reported, this does not work and fails with an exception and this is by design and thus we won't change it. -Matthias On 6/20/17 10:10 AM, Michal Borowiecki wrote: > Thanks, maybe I miss-read it: > >> I then tried implementing my own scheduling that periodically sends/clears >> out messages using the ProcessorContext provided to the aforementioned >> transform ste > I understood it to say it inspects a state store, sends the messages > that should be sent and removes them from the store. I might have read > too much out of it though. > > Cheers, > > MichaĆ > > > On 20/06/17 17:59, Matthias J. Sax wrote: >>>> I didn't know you could write to state stores from outside a >>>> processor/transformer. >> You can't. And as far as I understand this thread, nobody said you can. >> Did I miss something? >> >> >> -Matthias >> >> On 6/20/17 1:02 AM, Michal Borowiecki wrote: >>> I didn't know you could write to state stores from outside a >>> processor/transformer. Interesting to hear that it is working although >>> I'd be careful as KIP-67 warns it can introduce undefined behaviour: >>> >>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-67%3A+Queryable+state+for+Kafka+Streams >>> >>>> Operations on state stores >>>> >>>> The focus is on querying state stores, not updating them. It is not >>>> clear what it would mean to update a state store from outside the >>>> stream processing framework. Such updates are likely to introduce >>>> undefined behavior to the framework. >>>> >>> The way I'd approach it until KIP-138 is released is to still use >>> punctuate() but to also use your own scheduling to send periodic "tick" >>> messages into the input topic. These messages can be ignored by the >>> Processor but will cause the stream time to advance reliably. Just need >>> to ensure they are distributed uniformly to all partitions. >>> >>> I appreciate this is not a elegant workaround but this is what I've >>> settled for in the interim. >>> >>> Cheers, >>> >>> Michal >>> >>> >>> On 19/06/17 23:03, Steven Schlansker wrote: >>>>> On Jun 19, 2017, at 2:02 PM, Andre Eriksson <an...@tcell.io> wrote: >>>>> >>>>> I then tried implementing my own scheduling that periodically >>>>> sends/clears out messages using the ProcessorContext provided to the >>>>> aforementioned transform step. However, it seems that when I call >>>>> forward() from my scheduler (i.e. not in a process()/punctuate() call), I >>>>> get a NullPointerException at ProcessorContextImpl.java:81 >>>>> (https://github.com/apache/kafka/blob/0.10.2/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java#L81). >>>>> I assume that this is because currentNode() is null outside of >>>>> process()/punctuate() calls. >>>> There may be more elegant or direct solutions, but if all else fails you >>>> could always consider producing to a topic rather than trying to forward >>>> directly, then you don't have to touch the relatively delicate Processor >>>> semantics. >>>> >>> -- >>> Signature >>> <http://www.openbet.com/> Michal Borowiecki >>> Senior Software Engineer L4 >>> T: +44 208 742 1600 >>> >>> >>> +44 203 249 8448 >>> >>> >>> >>> E: michal.borowie...@openbet.com >>> W: www.openbet.com <http://www.openbet.com/> >>> >>> >>> OpenBet Ltd >>> >>> Chiswick Park Building 9 >>> >>> 566 Chiswick High Rd >>> >>> London >>> >>> W4 5XT >>> >>> UK >>> >>> >>> <https://www.openbet.com/email_promo> >>> >>> This message is confidential and intended only for the addressee. If you >>> have received this message in error, please immediately notify the >>> postmas...@openbet.com <mailto:postmas...@openbet.com> and delete it >>> from your system as well as any copies. The content of e-mails as well >>> as traffic data may be monitored by OpenBet for employment and security >>> purposes. To protect the environment please do not print this e-mail >>> unless necessary. OpenBet Ltd. Registered Office: Chiswick Park Building >>> 9, 566 Chiswick High Road, London, W4 5XT, United Kingdom. A company >>> registered in England and Wales. Registered no. 3134634. VAT no. >>> GB927523612 >>> > > -- > Signature > <http://www.openbet.com/> Michal Borowiecki > Senior Software Engineer L4 > T: +44 208 742 1600 > > > +44 203 249 8448 > > > > E: michal.borowie...@openbet.com > W: www.openbet.com <http://www.openbet.com/> > > > OpenBet Ltd > > Chiswick Park Building 9 > > 566 Chiswick High Rd > > London > > W4 5XT > > UK > > > <https://www.openbet.com/email_promo> > > This message is confidential and intended only for the addressee. If you > have received this message in error, please immediately notify the > postmas...@openbet.com <mailto:postmas...@openbet.com> and delete it > from your system as well as any copies. The content of e-mails as well > as traffic data may be monitored by OpenBet for employment and security > purposes. To protect the environment please do not print this e-mail > unless necessary. OpenBet Ltd. Registered Office: Chiswick Park Building > 9, 566 Chiswick High Road, London, W4 5XT, United Kingdom. A company > registered in England and Wales. Registered no. 3134634. VAT no. > GB927523612 >
signature.asc
Description: OpenPGP digital signature