Thanks.It would be good to add the concrete interfaces of the new classed to the KIP, ie,
- FixedKeyProcessorSupplier - FixedKeyProcessor - FixedKeyProcessorContext - FixedKeyRecord -Matthias On 3/10/22 3:15 PM, Jorge Esteban Quilcate Otoya wrote:
Thanks all! I agree with Matthias and Jon on going forward with the new `FixedKeyRecord` approach. The KIP has been updated accordingly. Feel free to add your vote or amend on the vote thread if needed. Cheers, On Mon, 7 Mar 2022 at 21:57, Matthias J. Sax <mj...@apache.org> wrote:I think it's ok that we cannot prevent users from mutating a given read-only object. We have similar issues "all over the place" in the API, because it's just how Java works unfortunately (eg, `ValueMapperWithKey` and similar interfaces). The point being is, that the API clearly expresses that the key should not be changes, as `FixedKeyRecord` as not `withKey()` method, what is much better then having `Record.withKey()` and thus incorrectly indicating to user that it would be ok to set a new key. I think it's worth to add the new interfaces. -Matthias On 3/7/22 11:46 AM, Guozhang Wang wrote:Thanks John! I feel a bit ashamed of just thinking loud here withouttryingout prototypes myself :P I think the FixedKeyProcessor/Record looks very good -- like you said, since we are making a new set of APIs then why don't we reconsider more bolderly -- but at the same time I'd also like to make sure we agree onhowmuch "safety" we can achieve in runtime: even with the proposed APIs, we cannot prevent users doing something like: --------------- process(FixedKeyRecord inputRecord) { inputRecord.key().modifyField(...); // this is not preventable with runtime key validations either since we just check the key object itselfisnot replaced context.forward(inputRecord); } --------------- I.e. in either type-safety or runtime validation, we cannot be 100% safe that users would not do anything wrong. This drives me to think, how much we'd like to pay to "remind" (instead of say "enforce", since we cannot really do it) users the semantics of "processValue". Personally I feltthatadding the new set of APIs for that purpose only is a bit overkill, and hence was leaning towards just the runtime validation. But I admit thisispurely subjective so I'm willing to yield to the group if others feelit'sworthy to do so. Guozhang On Mon, Mar 7, 2022 at 10:32 AM Jorge Esteban Quilcate Otoya < quilcate.jo...@gmail.com> wrote:Thanks, John! This looks very promising. I will familiarize this approach and update the KIP accordingly. FromwhatI can see so far, this should cover most of the open issues in this proposal. PS.Just as a reminder, the current approach with transformers is NOT enforced at compile time. Transformers have access to a "forwarding disabled" processor context, which still has the forward methods that throw a runtime exception when invoked.Agree. I was referring to the value transformers where `readOnlyKey` is passed but not forwarded internally. Though about the "forwardingdisabled"approach, you're totally right that is a runtime validation. Regardless, the approach proposed here will be a much better one. On Sun, 6 Mar 2022 at 18:59, John Roesler <vvcep...@apache.org> wrote:Hello all, It seems like we're making good progress on this discussion. If I'm keeping track correctly, if we can resolve this question about how to handle processValues(), then we should be able to finalize the vote, right? I share Matthias's preference for having a type-safe API. Just as a reminder, the current approach with transformers is NOT enforced at compile time. Transformers have access to a "forwarding disabled" processor context, which still has the forward methods that throw a runtime exception when invoked. However, the spirit of the "new processor api" line of work is to clean up a lot of the cruft around the original processor API, so this is a good opportunity to introduce a type-safe version if we can. Based on my experience adding the new processor API, I felt like it should be possible to do what he suggests, but it would be more involved than what he said. The biggest thing I learned from that effort, though, is that you really have to just try it to see what all the complications are. With that in mind, I went ahead and implemented the suggestion: https://github.com/apache/kafka/pull/11854 This is a functional prototype. It only adds processValues, which takes a supplier of a new type, FixedKeyProcessor. That processor only processes FixedKeyRecords, which have a key that cannot be changed. FixedKeyProcessors have a special context, a FixedKeyProcessorContext, which can only forward FixedKeyRecords. FixedKeyRecords have "fixed keys" because its key can only be set in the constructor, and its constructor is package- private. As you can see, this new record/processor/context ecosystem is an independent peer of the general one. This is necessary to ensure the desired compiler check. For example, if FixedKeyRecord were merely an interface implemented by Record, then users could create a new Record with a new key and forward it as a FixedKeyRecord, violating the constraint. As I said, with this proposal, the devil is in the details, so if anyone thinks the API can be simplified, I suggest you check out the branch and try out your proposal. I'd be very happy to have a simplier solution, but I'm also pretty sure this complexity is necessary. Taking a step back, I do think this approach results in a better API, even though the change is a little complicated. Thanks, -John On Sun, 2022-03-06 at 10:51 +0000, Jorge Esteban Quilcate Otoya wrote:Matthias, thanks for your feedback. I can see the following alternatives to deal with `processValues()`: 1. Runtime key validation (current proposal) 2. Using Void type. Guozhang already points out some important considerations about allocating `Record` twice. 3. Adding a new ValueRecord, proposed by Matthias. This one wouldcarrysome of the problems of the second alternative as ValueRecord willhavetobe created from a Record. Also, either by having a public constructororcreation from a Record, the key _can_ be changed without beingcapturedbythe Topology. 4. Reducing the KIP scope to `process` only, and removing/postponing `processValues` for a later DSL redesign. A couple of additional comments: About the Record API: IIUC, the issue with allocating new objects is coming from the current design of the Record API. If a user does record.withKey(...).withValue(...) is already leadingto acouple of instatiations. My impression is that if the cost/value of immutability has beenweighedalready, then maybe the considerations for alternative 2 can bedisregarded?Either way, if the cost of recreation of objects is something we wanttominimize, then maybe adding a Builder to the record should help toreducethe allocations. About the key validation: So far, the only way I can see to _really_ validate a key doesn'tchangeatcompile-time is by not exposing it at all — as we are doing it todaywithTransform. Otherwise, deal with it at runtime — as we have been dealing withTransformwithout the ability to forward. Processor API already —by definition— means lower-level abstraction, therefore users should be aware of the potential runtime exceptions ifthekey changes. This is why I'm leaning towards alternative 1. Looking forward to your feedback. As a reminder, the vote thread is still open. Feel free to add yourvoteoramend if needed. Cheers, On Fri, 4 Mar 2022 at 06:51, Matthias J. Sax <mj...@apache.org>wrote:John, thanks for verifying source compatibility. My impression wasthatit should be source compatible, I was just not 100% sure. The question about `processValues()` is really a hard one. Guozhang's point is very good one. Maybe we need to be pragmatic and accept the runtime check (even if I deeply hate this solution compare to acompiletime check). Other possibilities to address this issue might just become too ugly?Itseems it would require to add a new `ValueProcessorContext` thatoffersa `#forward(ValueRecord)` method (with `ValueRecord` being a `Record` with immutable key? Not sure if we would be willing to go down this route? Personally, I would be ok with it, as a strongly prefercompiletime checks and I am happy to extend the API surface area to achieveit-- however, I won't be surprised if others don't like this idea... -Matthias On 2/27/22 6:20 AM, Jorge Esteban Quilcate Otoya wrote:Thanks, Guozhang.Compared with reference checks and runtime exceptions for thosewhomistakenly change the key, I think that enforcing everyone to`setValue`may incur more costs..This is a fair point. I agree that this may incur in more coststhankeychecking. Will hold for more feedback, but if we agree I will update the KIPduringthe week. Cheers, Jorge. On Sun, 27 Feb 2022 at 00:50, Guozhang Wang <wangg...@gmail.com>wrote:Hello folks, Regarding the outstanding question, I'm actually a bit leaningtowardsthesecond option since that `withKey()` itself always creates a newRecordobject. This has a few implications: * That we would have to discard the previous Record object to beGC'edwiththe new object --- note in practice, processing value does notmeanyou'dhave to replace the whole value with `withValue`, but maybe youjustneedto manipulate some fields of the value object if it is a JSon /etc.* It may become an obstacle for further runtime optimizationse.g.skipserdes and interpret processing as direct byte manipulations. Compared with reference checks and runtime exceptions for thosewhomistakenly change the key, I think that enforcing everyone to`setValue`may incur more costs.. Guozhang On Fri, Feb 25, 2022 at 12:54 PM Jorge Esteban Quilcate Otoya < quilcate.jo...@gmail.com> wrote:Hi all, Appreciate very much all the great feedback received so far.After applying that interface change, I don't see any syntaxerrors in our tests (which use those methods), and the StreamBuilderTest still passes for me. This is awesome John, thank you for your efforts here.Jorge, do you mind clarifying these points in theCompatibilitysectionof your KIP? +1. I have clarified the impact of changing the return type inthe KIP.I think the other outstanding question for you is whether the output key type for processValues should be K or Void. One thing I realized belatedly was that if we do set it to Void, then users will actually have to override the key when forwarding, like `record.withKey(null)`, whereas if we keep it is K, all users have to do is not touch the key at all.This is a tricky one. On one hand, with Void type for key output, we force the usersto casttoVoid and change the key to null, though this can be documented on the API, so the users areawareof thepeculiarity of forwarding within `processValues`. On the other hand, keeping the key type as output doesn't_require_ todoany change of keys, but this could lead to key-checking runtime exceptions. I slightly inclined myself for the first option and change thetype to`Void`. This will impose a bit of pain on the users to gain sometype-safetyandavoid runtime exceptions. We can justify this requirement as a way to prove that the keyhasn'tchanged. Btw, thanks for this idea Matthias! On Fri, 25 Feb 2022 at 17:10, John Roesler <vvcep...@apache.org>wrote:Oh, one more thing Jorge, I think the other outstanding question for you is whether the output key type for processValues should be K or Void. I get the impression that all of us don't feel too strongly about it, so I think the ball is in your court to consider everyone's points and make a call (with justification). One thing I realized belatedly was that if we do set it to Void, then users will actually have to override the key when forwarding, like `record.withKey(null)`, whereas if we keep it as K, all users have to do is not touch the key at all. Thanks, -John On Fri, 2022-02-25 at 11:07 -0600, John Roesler wrote:Hello all, I'll chime in again in the interest of trying to do abetterjob of keeping KIPs moving forward... Matthias raised some very good questions about whether the change is really source compatible. I just checked out the code and make the interface change that Jorge specified in the KIP:Modified methods: KStream<KOut,VOut> KStream#process(ProcessorSupplier<K,V,KOut, VOut> processorSupplier, String... stateStoreNames)KStream<KOut,VOut> KStream#process(ProcessorSupplier<K,V,KOut, VOut> processorSupplier, Named named, String... stateStoreNames) After applying that interface change, I don't see anysyntaxerrors in our tests (which use those methods), and the StreamBuilderTest still passes for me. The reason is that the existing API already takes a ProcessorSupplier<K, V, Void, Void> and is currently a `void` return. After this interface change, all existing usages will just bind Void to KOut and Void to VOut. In other words, KOut, which is short for `KOut extends Object` is an upper bound on Void, so all existing processor suppliers are stillvalidarguments. Because the current methods are void returns, no existing code could be assigning the result to any variable, so moving from a void return to a typed return is always compatible. Jorge, do you mind clarifying these points in the Compatibility section of your KIP? Thanks, -John On Wed, 2022-02-23 at 15:07 -0800, Matthias J. Sax wrote:For this KIP, I also see the value. I was just trying tomake astepback and ask if it's a good short term solution. If webelieve itis,Iam fine with it. (I am more worried about the header's KIP...) Btw: I am still wondering if we can change existing`process()` asproposed in the KIP? It the propose change sourcecompatible? (It'sforsure not binary compatible, but this seems fine -- Idon'tthink weguarantee binary compatibility). Btw: would be good to clarify what is changes forprocess() --shouldbereturn type change from `void` to `KStream<KOut, VOut>`aswell aschange of `ProcessorSupplier` generic types (output typeschangefrom`Void` to `KOut` and `VOut`? -Matthias On 2/23/22 11:32 AM, Guozhang Wang wrote:Hi folks, I agree with John that this KIP by itself could be agoodimprovement, andI feel it aligns well with the eventual DSL 2.0proposalso we donot needto hold it until later. Regarding the last point (i.e. whether we should doenforcementwitha newinterface), here's my 2c: in the past we introducedpublic`ValueTransfomer/etc` for two purposes, 1) to enforcethe key isnotmodifiable, 2) to indicate inside the library'stopologybuilderitselfthat since the key is not modified, the directdownstream doesnotneed toinject a repartition stage. I think we are more or lesson thesamepagethat for purpose 1), doing runtime check could besufficient; asforthepurpose of 2), as for this KIP itself I think it issimilar towhatwe have(i.e. just base on the function name "processValue"itself) andhence arenot sacrificed either. I do not know if `KStream#processValue(ProcessorSupplier<K, V, Void,VOut>processorSupplier)` will work, or work better, maybeJorge coulddosomedigging and get back to us. On Fri, Feb 18, 2022 at 8:24 AM John Roesler <vvcep...@apache.org>wrote:Hello all, While I sympathize with Matthias’s desire to wipe theslatecleanandredesign the dsl with full knowledge of everythingwe’velearnedin thepast few years, that would also be a pretty intenseproject onitsown. Itseems better to leave that project for someone who ismotivatedtotake iton. Reading between the lines, it seems like Jorge’smotivation ismore alongthe lines of removing a few specific pain points. IappreciateMatthiasextending the offer, but if Jorge doesn’t want toredesign thedslrightnow, we’re better off just accepting the work he’swilling todo.Specifically, this KIP is quite a nice improvement.Looking attheKStreaminterface, roughly half of it is devoted to variousflavors of“transform”,which makes it really hard on users to figure outwhich theyaresupposedto use for what purpose. This kip let us drop allthatcomplexityin favorof just two methods, thanks to the fact that we nowhave theability forprocessors to specify their forwarding type. By the way, I really like Matthias’s suggestion tosetthe KOutgenericbound to Void for processValues. Then, instead ofdoing anequality checkon the key during forward, you’d just set the keybackto theonesavedbefore processing (with setRecordKey). This is bothmoreefficient(becausewe don’t have the equality check) and more foolprooffor users(becauseit’s enforced by the compiler instead of theruntime).Thanks, all! -John On Fri, Feb 18, 2022, at 00:43, Jorge EstebanQuilcateOtoyawrote:On Fri, 18 Feb 2022 at 02:16, Matthias J. Sax <mj...@apache.org>wrote:It probably deserves its own thread to startdiscussingideas.Yes. My question was: if we think it's time to doa DSL2.0,should wedrop this KIP and just fix via DSL 2.0 instead?Good question. Would love to hear what others thinkaboutthis.I've stated my position about this here:For this KIP specifically, I think about it as acontinuationfromKIP-478. Therefore, it could make sense to have itas part ofthe currentversion of the DSL. I'd even add that if this KIP is adopted, I wouldnot be thatdisappointedif KIP-634 is dropped in favor of a DSL v2.0 as theaccess toheadersprovided by KIP-478's via Record API is much betterthanprevious`.context().headers()`. But happy to reconsider if there is an agreement tofocuseffortstowards aDSL 2.0.You're right. I'm not proposing the methodsignature.What signature do you propose? I don't see anupdate on theKIP.My bad. I have clarified this in the KIP's publicinterfacesnow:``` New methods: - KStream<K,VOut>KStream#processValues(ProcessorSupplier<K,V, K,VOut>processorSupplier, String... stateStoreNames) - KStream<K,VOut>KStream#processValues(ProcessorSupplier<K,V, K,VOut>processorSupplier, Named named, String...stateStoreNames)Modified methods: - KStream<KOut,VOut>KStream#process(ProcessorSupplier<K,V,KOut,VOut>processorSupplier, String... stateStoreNames) - KStream<KOut,VOut>KStream#process(ProcessorSupplier<K,V,KOut,VOut>processorSupplier, Named named, String...stateStoreNames)```Not sure if I understand how this would looklike. Do youmeancheckingiton the Record itself or somewhere else?@Guozhang: I am not worried about the runtimeoverhead. Iamworriesabout user experience. It's not clear from themethodsignature, thatyou are not allowed to change the key, what seemsto be badAPI desig.Even if I understand the desire to keep the APIsurfacearessmall -- Iwould rather have a compile time enforcement thana runtimecheck.For example, we have `map()` and `mapValues()`and`mapValues()` returnsa `Value V` (enforces that that key is notchanges) insteadofa`KeyValue<KIn,VOut>` and we use a runtime checktocheckthatthe key isnot changed. Naively, could we enforce something similar bysetting theoutput keytype as `Void`. KStream#processValue(ProcessorSupplier<K, V,Void,VOut>processorSupplier) Not sure if this would work or not? Or it might be worth to add a new interface,`ValueProcessorSupplier`that ensures that the key is not modified?This is an important discussion, even more so withaDSLv2.0.At the moment, the DSL just flags whetherpartitioning isrequired basedonthe DSL operation. As mentioned, `mapValues()`enforces onlythevaluehaschanged through the DSL, though the only_guarantee_we haveisthatKafkaStreams "owns" the implementation, and we can flagthisproperly.With a hypothetical v2.0 based on Record API, thiswill beharder toenforce with the current APIs. e.g. with`mapValues(Record<K,V>record)`,nothing would stop users from using`record.withKey("needs_partitioning")`.The approach defined on this KIP is similar to whatwe haveatthe momenton `ValueTransformer*` where it validates atruntimethat theusers arenotcalling `forward` with`ForwardingDisabledProcessorContext`.`ValueProcessorSupplier` is not meant to be apublicAPI.Onlyto be usedinternally on `processValues` implementation. At first,`KStream#processValue(ProcessorSupplier<K,V, Void,VOut>processorSupplier)` won't work as it will requirethe`Processor`implementation to actually change the key. Willtakea deeperlook tovalidate if this could solve this issue.-Matthias On 2/17/22 10:56 AM, Guozhang Wang wrote:Regarding the last question Matthias had, Iwonder ifit'ssimilar tomyfirst email's point 2) above? I think therationale isthat,sincereference checks are relatively very cheap, itisworthwhileto paythisextra runtime checks and in return to have asingleconsolidatedProcessorSupplier programming interface (i.e.wewouldeventuallydeprecate ValueTransformerWithKeySupplier). On Wed, Feb 16, 2022 at 10:57 AM Jorge EstebanQuilcateOtoya <quilcate.jo...@gmail.com> wrote:Thank you Matthias, this is great feedback. Adding my comments below. On Wed, 16 Feb 2022 at 00:42, Matthias J.Sax <mj...@apache.org>wrote:Thanks for the KIP. In alignment to my reply to KIP-634, I amwonderingifwe areheadinginto the right direction, or if we shouldconsider tore-design theDSLfrom scratch?I'm very excited about the idea of a DLSv2.0.Itprobablydeservesitsownthread to start discussing ideas. For this KIP specifically, I think about itasacontinuation fromKIP-478.Therefore, it could make sense to have it aspart ofthecurrentversion ofthe DSL.Even if we don't do a DSL 2.0 right now, Ihave someconcerns aboutthisKIP: (1) I am not sure if the propose changed isbackwardcompatible? Wecurrently have: voidKStream#process(ProcessorSupplier,String...)The newly proposed method: KStreamKStream#process(ProcessorSupplier)seems to be an incompatible change? The KIP states:Modified method KStream#process should becompatiblewith previousversion, that at the moment is fixed to aVoid returntype.Why is it backward compatible? Having bothold andnew#process()seemsnot to be compatible to me? Or are youproposing to_change_ themethodsignature (if yes, the `String...`parameterto add astate storeseemsto be missing)? For this case, it seemsthatexistingprogramswould atleast need to be recompiled -- it wouldonlybe asourcecompatiblechange, but not a binary compatible change?You're right. I'm not proposing the methodsignature.Totally agree about compatibility issue. Iwasonlyconsideringsourcecompatibility and was ignorant that changingfrom voidtoa specifictypewould break binary compatibility. I will update the KIP to reflect this:Modifications to method KStream#process aresourcecompatible withprevious version, though not binarycompatible.Thereforewillrequireusers to recompile their applications withthelatestversion.I am also wondering if/how this changerelated toKIP-401:https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=97553756From a high level it might notconflict,but Iwantedto doublecheck?Wasn't aware of this KIP, thanks for sharing!I don'tthink there isconflict between KIPs, as far as Iunderstand.For `KStream#processValues()`, my mainconcern is theadded runtimecheck if the key was modified or not -- itseems toprovide bad userexperience -- enforcing that the key is notmodifiedonan APIlevel,would seem to be much better. Last, what is the purpose of`setRecordKey()` and`clearRecordKey()`? Iam not sure if I understand their purpose?Both methods set/clear the context (currentkey) to beused whencheckingkeys on forward(record) implementation.enforcing that the key is not modified onanAPIlevel,would seemtobemuch better. Not sure if I understand how this would looklike. Doyoumeancheckingiton the Record itself or somewhere else?-Matthias On 2/15/22 11:53 AM, John Roesler wrote:My apologies, this feedback was intendedforKIP-634.-John On Tue, Feb 15, 2022, at 13:15, JohnRoesler wrote:Thanks for the update, Jorge, I've just looked over the KIP again.Just onemoresmallconcern: 5) We can't just change the type ofRecord#headers()to anew fully qualified type. That would bea source-incompatible breaking change for users. Out options are: * Deprecate the existing method andcreate a newonewiththe new type * If the existing Headers is "not greatbut ok",then maybewe leave it alone. Thanks, -John On Mon, 2022-02-14 at 13:58 -0600, PaulWhalenwrote:No specific comments, but I justwanted tomentionI like thedirection ofthe KIP. My team is a big user of"transform"methods because oftheability to chain them, and I havealways foundtheterminologychallengingto explain alongside "process". Itfelt likeoneconcept withtwonames.So moving towards a single API thatispowerfulenough to handlebothusecases seems absolutely correct to me. Paul On Mon, Feb 14, 2022 at 1:12 PM JorgeEstebanQuilcate Otoya <quilcate.jo...@gmail.com> wrote:Got it. Thanks John, this makesense.I've updated the KIP to include thedeprecationof:- KStream#transform - KStream#transformValues - KStream#flatTransform -KStream#flatTransformValuesOn Fri, 11 Feb 2022 at 15:16, JohnRoesler <vvcep...@apache.orgwrote:Thanks, Jorge! I think it’ll be better to keepthis KIPfocused on KStreammethodsonly.I suspect that the KTable methodsmay bemorecomplicated thanjustthatproposed replacement, but it’llalso beeasierto consider thatquestioninisolation. The nice thing about justdeprecating theKStream methods andnottheTransform* interfaces is that youcan keepyour proposal justscopedtoKStream and not have anyconsequences fortherest of the DSL.Thanks again, John On Fri, Feb 11, 2022, at 06:43,JorgeEstebanQuilcate Otoyawrote:Thanks, John.4) I agree that we shouldn'tdeprecatetheTransformer*classes, but do you think weshoulddeprecate theKStream#transform* methods? I'mcuriousifthere's anyremaining reason to have thosemethods,orif your KIPcompletely obviates them. Good catch. I considered that deprecating`Transformer*`and `transform*`wouldgohandin hand — maybe it happenedsimilarlywithold `Processor` and`process`?Though deprecating only`transform*`operations could be abettersignalfor users than non deprecatinganythingatall and pave thewaytoit'sdeprecation. Should this deprecation alsoconsiderincluding`KTable#transformValues`?The approach proposed on theKIP:`ktable.toStream().processValues().toTable()` seems fair tome,thoughIwill have to test it further. I'm happy to update the KIP ifthere'ssomeconsensus aroundthis.Will add the deprecation notesthese daysand wait for anyadditionalfeedback on this topic beforewrapping upthe KIP.On Fri, 11 Feb 2022 at 04:03,JohnRoesler<vvcep...@apache.org>wrote:Thanks for the update, Jorge! I just read over the KIPagain, and I'minsupport. One morequestion came up for me,though:4) I agree that we shouldn'tdeprecatetheTransformer*classes, but do you think weshoulddeprecate theKStream#transform* methods?I'm curiousifthere's anyremaining reason to havethosemethods,orif your KIPcompletely obviates them. Thanks, -John On Thu, 2022-02-10 at 21:32+0000,JorgeEsteban QuilcateOtoya wrote:Thank you both for yourfeedback!I have added the followingnote onpunctuation:``` NOTE: The key validationcanbedefinedwhen processing themessage.Though, with punctuationsitwon't bepossible to define thekeyforvalidation beforeforwarding,thereforeit won't bepossible toforwardfrom punctuation. This is similar behavior tohow`ValueTransformer`s behaveatthemoment.``` Also make it explicit alsothat wearegoing to applyreferencialequalityfor key validation. I hope this is covering allyourfeedback, let me know ifI'mmissinganything. Cheers, Jorge. On Wed, 9 Feb 2022 at22:19,GuozhangWang <wangg...@gmail.comwrote:I'm +1 on John's point 3)forpunctuations.And I think if people areon thesamepage that a referenceequalitycheckper record is not a hugeoverhead,Ithink doing thatenforcementisbetterthan documentations andhand-wavyundefined behaviors.Guozhang On Wed, Feb 9, 2022 at11:27 AMJohnRoesler <vvcep...@apache.orgwrote:Thanks for the KIPJorge,I'm in support of yourproposal.1) I do agree withGuozhang's point(1). I think the cleanestapproach. I think it'scleanerandbetter to keep theenforcement internal totheframework than to introduce apublic API or contextwrapper forprocessors to useexplicitly. 2) I tend to agree withyou onthisone; I think theequality check ought tobe fastenough in practice.3) I think this isimplicit, butshould be explicit in theKIP: For the`processValues` API,because the frameworksetsthe key on the contextbeforecalling `process` and thenunsets it afterwards,there willalways be no key setduringtask puctuation.Therefore, whileprocessors may stillregister punctuators,they willnotbe able to forwardanything from them. This is functionallyequivalenttothe existingtransformers, by theway, thatarealso forbidden toforwardanything duringpunctuation.For what it's worth, Ithink thisisthe best tradeoff.The only alternative Isee is nottoplace any restrictionon forwarded keys atalland justdocument that if usersdon't maintain properpartitioning,they'll get undefinedbehavior. That might bemorepowerful, but it's also ausability problem. Thanks, -John On Wed, 2022-02-09 at11:34+0000,Jorge Esteban QuilcateOtoya wrote:Thanks Guozhang.Does`ValueProcessorContext`have to be a public API? Itseemsto methat this can becompletelyabstracted away from userinterfacesas aninternal class Totally agree. Nointention toaddthese as public APIs.WillupdatetheKIP to reflect this.in the past therationale forenforcing it at theinterface layerratherthan doruntime checks is that itismoreefficient.I'm not sure howmuchoverheadit may incur to check ifthekeydidnotchange: if it isjust areferenceequality check maybeit'sokay.What'syour take on this? Agree, referenceequalityshouldcover this validationandtheoverheadimpact should not bemeaningful.Will update the KIPtoreflectthis as well.On Tue, 8 Feb 2022 at19:05,Guozhang Wang <wangg...@gmail.com>wrote:Hello Jorge, Thanks for bringingthisKIP! Ithink this is a niceideatoconsiderusinga single overloadedfunctionname for #process, just acouplequickquestions afterreading theproposal:1) Does`ValueProcessorContext`have to be a publicAPI? Itseems tomethat this can becompletelyabstracted away from userinterfacesasaninternal class, andwe callthe`setKey` before callinguser-instantiated`process` function,and theninits overridden`forward` itcanjustcheckif the key changesor not.2) Related to 1)above, inthepast the rationale forenforcingit attheinterface layerrather thandoruntime checks is thatit ismoreefficient.I'm not sure howmuchoverheadit may incur to check ifthekeydidnotchange: if it isjust areference equality check maybeit'sokay.What'syour take on this? Guozhang On Tue, Feb 8, 2022at 5:17AMJorge Esteban QuilcateOtoya<quilcate.jo...@gmail.com>wrote:Hi Dev team, I'd like to starta newdiscussion thread on KafkaStreamsKIP-820:https://cwiki.apache.org/confluence/display/KAFKA/KIP-820%3A+Extend+KStream+process+with+new+Processor+APIThis KIP is aimedto extendthe current`KStream#process`APItoreturnoutput valuesthatcould bechained across thetopology,aswell asintroducing a new`KStream#processValues` to useprocessorwhilevalidatingkeys haven'tchange andrepartition is not required.Looking forwardtoyourfeedback.Regards, Jorge.-- -- Guozhang-- -- Guozhang-- -- Guozhang