Re: [DISCUSS] KIP-747 Add support for basic aggregation APIs

2021-08-16 Thread Mohan Parthasarathy
pile... Or would there be a simpler way to ensure that min() can only > >> be called _if_ V is `Comparable`? > >> > >> > >> So maybe passing in a `Comparator` might be the right way to go; > >> might also be more flexible anyway. -- My original idea was jus

Re: [DISCUSS] KIP-747 Add support for basic aggregation APIs

2021-08-08 Thread Mohan Parthasarathy
ot;Double" and you had "Long" in one of your responses I think. Do you have a preference or reason one way or other ? -mohan > -Matthias > > On 8/8/21 7:41 AM, Mohan Parthasarathy wrote: > > On Tue, Aug 3, 2021 at 6:56 PM Matthias J. Sax wrote: > > > >&g

Re: [DISCUSS] KIP-747 Add support for basic aggregation APIs

2021-08-08 Thread Mohan Parthasarathy
sure why we would want to pass `Function> > func` into `min()`? > > Not sure. Also, do you have an opinion on Long vs Double ? -thanks Mohan > > > -Matthias > > > > On 6/21/21 11:23 AM, Mohan Parthasarathy wrote: > > Alex, > > > > > > On Wed,

Re: [DISCUSS] KIP-501 Avoid out-of-sync or offline partitions when follower fetch requests are not processed in time

2021-06-28 Thread Mohan Parthasarathy
ting the first option to true does not necessarily mean that the condition is happening. Did you mean to say that relinquish the leadership if it is taking longer than leader.fetch.process.time.max.ms AND there are fetch requests pending which are >= log-end-offset of the earlier fetch reque

Re: [DISCUSS] KIP-501 Avoid out-of-sync or offline partitions when follower fetch requests are not processed in time

2021-06-27 Thread Mohan Parthasarathy
Hi Satish, One small clarification regarding the proposal. I understand how Solution (1) enables the other replicas to be chosen as the leader. But it is possible that the other replicas may not be in sync yet and if unclean leader election is not enabled, the other replicas may not become the lea

Re: [DISCUSS] KIP-655: Windowed "Distinct" Operation for KStream

2021-06-24 Thread Mohan Parthasarathy
Ivan, I read through the discussion and your new proposal. I have a couple of questions. 1. As we have cancelRepartition, wouldn't selectKey be sufficient. You still have idExtractor. Maybe I misunderstood the discussion. 2. isPersistent should be replaced by Materialized. It looked like you agre

Re: [DISCUSS] KIP-747 Add support for basic aggregation APIs

2021-06-21 Thread Mohan Parthasarathy
require a ` > Number>`. > > Are there any reasons to limit the sum() to integers? Why not use a Double > instead? > > Yeah, if the precision is important, then we should stick with Double. -mohan Best regards, > Alexandre > > On Wed, Jun 16, 2021 at 1:01 AM Mohan Parth

Re: [DISCUSS] KIP-747 Add support for basic aggregation APIs

2021-06-15 Thread Mohan Parthasarathy
ke sense though, and also to require a ` Number>`. > > I guess these are the two possibilities: Long sum(Function func) Long sum(Function func) Both should work. "func" can return any subtypes of Number and I don't see any advantages with the first version. Can you clarify

Re: [DISCUSS] KIP-747 Add support for basic aggregation APIs

2021-06-08 Thread Mohan Parthasarathy
thods in this KIP would be a syntactic sugar for what could > otherwise be implemented > using reduce/aggregate, but I see value in implementing them and > simplifying the adoption of > those use cases. > > Agreed. I seem to have forgotten the reason as to why I started this KIP :

Re: [DISCUSS] KIP-747 Add support for basic aggregation APIs

2021-06-05 Thread Mohan Parthasarathy
; is called the first time OR infer from the materialized or have an explicit initializer. Thanks Mohan > [1]: An example scenario for this would be to find the min / max Bidding > for a product where, at the end of the > auction, I need not only the min / max value of said Bidding, but

[DISCUSS] KIP-747 Add support for basic aggregation APIs

2021-06-02 Thread Mohan Parthasarathy
Hi, I have created a proposal for adding some additional aggregation APIs like count. https://cwiki.apache.org/confluence/display/KAFKA/KIP-747+Add+support+for+basic+aggregation+APIs I have noted down some of the issues that need discussion. Thanks to Matthias for helping me with the scope of th

[jira] [Resolved] (KAFKA-8214) Handling RecordTooLargeException in the main thread

2019-04-11 Thread Mohan Parthasarathy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-8214?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Mohan Parthasarathy resolved KAFKA-8214. Resolution: Duplicate > Handling RecordTooLargeException in the main thr

[jira] [Created] (KAFKA-8214) Handling RecordTooLargeException in the main thread

2019-04-10 Thread Mohan Parthasarathy (JIRA)
Mohan Parthasarathy created KAFKA-8214: -- Summary: Handling RecordTooLargeException in the main thread Key: KAFKA-8214 URL: https://issues.apache.org/jira/browse/KAFKA-8214 Project: Kafka