No specific comments, but I just wanted to mention I like the direction of
the KIP. My team is a big user of "transform" methods because of the
ability to chain them, and I have always found the terminology challenging
to explain alongside "process". It felt like one concept with two names.
So
Whoops, please ignore. I didn't pay enough attention to my autocomplete
for Apache mailing lists :)
On Wed, Apr 21, 2021 at 11:37 AM Paul Whalen wrote:
> Hi all,
>
> I'm using the Go Flight client, working off of this example:
> https://github.com/apache/arrow/blob/master/go/
Hi all,
I'm using the Go Flight client, working off of this example:
https://github.com/apache/arrow/blob/master/go/arrow/flight/flight_test.go
I've found that there isn't a very convenient way to access the app
metadata from a Flight stream, because the ipc.Reader you get from calling
Nikolay,
I'm not a committer, but perhaps I can start the discussion. I've had the
urge for a similar feature after being bitten by writing a poorly formed
record to a topic - it's natural to want to push schema validation into the
broker, since that's the way regular databases work. But I'm a
o there’s a good chance that
> >> real
> >> > > applications in use are depending on undefined context without even
> >> > > realizing it. What I’m hoping to do is just make the situation
> >> explicit and
> >> > > get rid of the dummy values.
>
this case should use
> > > > TimestampedKeyValueStore. The passed timestamp will be set on the
> > > > changelog records for this case.
> > > >
> > > > Thus, for both cases, accessing the record context does not seems to
> be
> > > > a requirement. An
; > > > > > >
> > > > > > > > Funny story, I also started down this road a couple of times
> > > > > > > > already and backed them out before the KIP because I was
> > > > > > > > afraid of the scope of the proposal. U
John,
It's exciting to see this KIP head in this direction! In the last year or
so I've tried to sketch out some usability improvements for custom state
stores, and I also ended up splitting out the StateStoreContext from the
ProcessorContext in an attempt to facilitate what I was doing. I sort
Perhaps I’m misunderstanding, but this looks like the cogroup feature:
https://cwiki.apache.org/confluence/display/KAFKA/KIP-150+-+Kafka-Streams+Cogroup.
Do you think that covers your use case?
Paul
> On Jun 10, 2020, at 10:13 PM, lqjacklee wrote:
>
> Dear team,
>
> I have created the
[
https://issues.apache.org/jira/browse/KAFKA-8177?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Paul Whalen resolved KAFKA-8177.
Resolution: Resolved
> Allow for separate connect instances to have sink connectors with the s
+1 (non-binding). I haven’t contributed to the discussion but I’ve been
following - it’ll definitely make my team’s life easier.
> On Sep 20, 2019, at 11:36 AM, Jukka Karvanen
> wrote:
>
> Hi all,
>
> I would like to start vote on KIP-470:
>
Manikumar,
KIP-401 was accepted a few weeks ago and there is a PR pending review, can
it be included in the release as well?
Thanks,
Paul
On Mon, Sep 16, 2019 at 6:14 AM Manikumar wrote:
> Hi All,
>
> Just a reminder that any new/pending KIP must pass vote by next Wednesday
> (Sep 25, 2019)
With 3 binding votes and 1 non-binding, the vote is closed and the KIP is
accepted.
I'm just wrapping up a first draft of a PR here:
https://github.com/apache/kafka/pull/6824
Thanks!
Paul
On Thu, Sep 5, 2019 at 2:17 PM Paul Whalen wrote:
> Thanks all! I updated the KIP status, and will
t
> as accepted.
>
>> On Tue, Sep 3, 2019 at 3:17 PM Paul Whalen wrote:
>>
>> Yeah, agreed on it being the same reference. That’s the way I have it in
>> the working PR and I’ll update the KIP for clarity.
>>
>>>> On Sep 3, 2019, at 5:04 PM, Matth
toreBuilder instances are
>> the same: either 1) equality by reference or 2) equality based on e.g.
>> #equals override function so that two different instances may still be
>> considered "equal". I think you meant 1), just wanted to confirm :)
>>
>>
>> Guo
Roesler wrote:
>>> I'm +1 (nonbinding) on the current iteration of the proposal.
>>>
>>>> On Mon, May 27, 2019 at 1:58 PM Paul Whalen wrote:
>>>>
>>>> I spoke too early a month ago, but I believe the proposal is finalized
>> now
>>>
I updated the KIP (and PR) to relax the restriction on connecting state
stores via either means; it definitely makes sense to me at this point.
I'd love to hear if there are any other concerns or broad objections to the
KIP.
Paul
On Thu, Aug 8, 2019 at 10:12 PM Paul Whalen wrote:
> Matth
e it idempotent what seems hard
> to achieve, because both `ProcessorSuppliers` now have a cross
> dependency to us the same object.
>
> Hence, I don't think this would be a good approach.
>
>
> Also, because we require for a unique store name to always pass the same
e for Streams to
> > connect, rather than that they are already connected, you could call
> > it ConnectableStoreProvider (similar to AutoCloseable).
> >
> > I just thought I'd summarize the current state, since it's been a
> > while and no one has voted yet. I'll go
First of all, +1 on the whole idea, my team has run into (admittedly minor,
but definitely annoying) issues because of the weaker typing. We're heavy
users of the PAPI and have Processors that, while not hundreds of lines
long, are certainly quite hefty and call context.forward() in many places.
I've only skimmed it so far, but great job! The community is in serious
need of more examples of the Processor API, there really isn't that much
out there.
One thing I did notice: the iterator you get from kvStore.all() ought to be
closed to release resources when you're done with it. This
It’s not totally clear, but this may be
https://issues.apache.org/jira/plugins/servlet/mobile#issue/KAFKA-7941
For which there is a fix that is very nearly approved:
https://github.com/apache/kafka/pull/6283
Paul
> On Jun 5, 2019, at 1:26 AM, Srinivas, Kaushik (Nokia - IN/Bangalore)
>
@%3Cdev.kafka.apache.org%3E
Pull request (still a WIP, obviously):
https://github.com/apache/kafka/pull/6824
Thanks,
Paul
On Wed, Apr 24, 2019 at 8:00 PM Paul Whalen wrote:
> Hi all,
>
> After some good discussion on and adjustments to KIP-401 (which I renamed
> slightly for clarity), chatter has
thread about a month ago, so I'll bump that now that
we're nearly there.
Paul
On Sun, May 26, 2019 at 2:21 PM Paul Whalen wrote:
> Per Matthias's suggestion from a while ago, I actually implemented a good
> amount of option B to get a sense of the user experience and documentation
> req
ns, if we choose to not provide a default implementation and let
> > the compiler signal the necessity to override the method, we have to
> > separate the interfaces in any case.
> >
> > Ad 2)
> > If you check for `null` or empty list in `process` and do not call
> >
Atm, `split()` and `branch()` would return `BranchedKStream`
>> and the call to `defaultBranch()` that returns the `Map` is mandatory
>> (what is not the case atm). Or is this actually not a real problem,
>> because users can just ignore the branch returned by `defaultBranch()`
>>
s downstream
> functionality. Programs that have deep branch trees will quickly become
> harder to read as a single unit.
>
>> On Tue, Apr 30, 2019 at 8:34 PM Paul Whalen wrote:
>>
>> Also +1 on the issues/goals as Michael outlined them, I think that sets
.
> >
> > If `Transformer extends StoreProvider` (just picking a name here)
> > without default implementation existing code would break and thus it not
> > a an option because of breaking backward compatibility.
> >
> >
> > -Matthias
> >
> > On 4/
gt; > > > > (https://github.com/apache/kafka/pull/6512)
> > > > >
> > > > > Any new suggestions/objections?
> > > > >
> > > > > Regards,
> > > > >
> > > > > Ivan
> > > > >
> > &
to detect bugs in user code for which, per accident,
> an empty collection could be returned.
>
>
>
> (4) Should the new method return a `Set` instead of a `Collection` to
> indicate the semantics clearly (ie, returning the same `StoreBuilder`
> multiple times is idemp
Hi all,
After some good discussion on and adjustments to KIP-401 (which I renamed
slightly for clarity), chatter has died down so I figured I may as well
start a vote.
KIP:
TransformerSupplier/ProcessorSupplier StateStore connecting
t as it does not require another
> `addStore` call, but we just need to spend some more documentation effort
> on educating users about the two ways of connecting their stores. I'm
> slightly concerned about this education curve but I can be convinced if
> most people felt it is worthy.
his direction'? To me it looks like a good starting point. But
> as a novice in this project I might miss some important details.
>
> Regards,
>
> Ivan
>
>
> 28.03.2019 17:38, Paul Whalen пишет:
> > Ivan,
> >
> > Maybe I’m missing the point, but I believe the str
Paul Whalen created KAFKA-8177:
--
Summary: Allow for separate connect instances to have sink
connectors with the same name
Key: KAFKA-8177
URL: https://issues.apache.org/jira/browse/KAFKA-8177
Project
tion!!!*/
> couponIssuer.coupons()...
>
> Does this make sense? In order to properly initialize the CouponIssuer we
> need the terminal operation to be called before streamsBuilder.build() is
> called.
>
>
> [BTW Paul, I just found out that your KIP-401 is essentially the next KIP I
>
would be fairly easily for the InternalTopologyBuilder to track
> >>> dangling
> >> branches that haven't been terminated and raise a clear error before it
> >> becomes an issue.
> >>
> >> You mean a runtime exception, when the program is compiled a
elete the
> whole test dataset.
>
>
> Does it seem like you have a good path forward? From what I'm
> hearing, the "user-space" approach is at least worth exploring before
> considering a new API. Of course, if it doesn't pan out for whatever
> reason,
> I'd (perso
ou could do this with external tooling, e.g. a script
> leveraging RemoteClusterUtils and kafka-streams-application-reset.sh. I
> haven't tried this with a Streams app myself, but I suspect it would work.
>
> Ryanne
>
>
> On Sun, Mar 24, 2019 at 12:31 PM Paul Whalen wrote:
>
>
ect construction
> contrasts the fluency of most KStream methods. But here we have a
> special case: we build the switch to split the flow, so I think this is
> still idiomatic.
>
> Regards,
>
> Ivan
>
>
>
> 24.03.2019 4:02, Paul Whalen пишет:
> > Ivan,
Hi all,
With MirrorMaker 2.0 (
https://cwiki.apache.org/confluence/display/KAFKA/KIP-382%3A+MirrorMaker+2.0)
accepted and coming along very nicely in development, it has got me
wondering if a certain use case is supported, and if not, can changes be
made to Streams or MM2 to support it. I'll
Ivan,
I think it's a great idea to improve this API, but I find the onTopOff()
mechanism a little confusing since it contrasts the fluency of other
KStream method calls. Ideally I'd like to just call a method on the stream
so it still reads top to bottom if the branch cases are defined fluently.
pe to shared state stores only,; hence users need to
> learn two ways of creating state stores for those two patterns.
>
> My argument is that more public APIs requires longer learning curve for
> users, and introduces more usage patterns that may confuse users (the
> proposal I had trie
Paul Whalen created KAFKA-7941:
--
Summary: Connect KafkaBasedLog work thread terminates when getting
offsets fails because broker is unavailable
Key: KAFKA-7941
URL: https://issues.apache.org/jira/browse/KAFKA-7941
+1 non binding. I haven't contributed at all to discussion but have
followed since Adam reinvigorated it a few months ago and am very excited
about it. It would be a huge help on the project I'm working on.
On Fri, Jan 11, 2019 at 9:05 AM Adam Bellemare
wrote:
> Thanks all -
>
> So far that's
nd that must be added first to the
> >> topology. The other transformers would implement `StoreNameSupplier` and
> >> just connect to those stores.
> >>
> >> Another possibility to avoid the issue of adding the same stores
> >> multiple times would be, that the
s
> on accessing the store from different operators within the same task. I'm
> not sure how common this use case is, but I'd like to hear if you have any
> thoughts maintaining this since the current proposal seems exclude this
> possibility.
>
>
> Guozhang
>
>
>
Here's KIP-401 for discussion, a minor Kafka Streams API change that I
think could greatly increase the usability of the low-level processor API.
I have some code written but will wait to see if there is buy in before
going all out and creating a pull request. It seems like most of the work
would
Based on some other messages the mailing list seems like this is the best
place to ask for permissions. Can it be granted on the wiki for pgwhalen?
Gonna try to write something up for this:
https://issues.apache.org/jira/browse/KAFKA-7523
Thanks!
Paul
Paul Whalen created KAFKA-7523:
--
Summary: TransformerSupplier/ProcessorSupplier enhancements
Key: KAFKA-7523
URL: https://issues.apache.org/jira/browse/KAFKA-7523
Project: Kafka
Issue Type
49 matches
Mail list logo