Re: [DISCUSS] KIP-63: Unify store and downstream caching in streams

2016-06-03 Thread Eno Thereska
Just to clarify: "cache.max.bytes.buffering" is per processor? > > > On Thu, Jun 2, 2016 at 11:30 AM, Eno Thereska <eno.there...@gmail.com> wrote: >> Hi there, >> >> I have created KIP-63: Unify store and downstream caching in streams >> https:/

Re: Kafka Streams aggregation store

2016-06-08 Thread Eno Thereska
Hi Alexander, I haven't tried Kafka Streams on Windows but did notice that Microsoft has merged code into github to make RocksDB available on Windows. Perhaps this is useful:

Re: [DISCUSS] KIP-63: Unify store and downstream caching in streams

2016-06-06 Thread Eno Thereska
ectSize which is approximate, it > may exaggerate the imbalance. > > > Guozhang > > > On Sat, Jun 4, 2016 at 11:54 PM, Eno Thereska <eno.there...@gmail.com> > wrote: > >> Hi Jay, >> >> We can make it global instead of per-processor, sounds goo

Re: Kafka Streaming - Window expiration

2016-06-13 Thread Eno Thereska
Hi Pari, Try the .until method like this: > (TimeWindows) TimeWindows.of("tumbling-window-example", > windowSizeMs).until(60 * 1000L) Thanks Eno > On 13 Jun 2016, at 08:31, Pariksheet Barapatre wrote: > > Hello Experts, > > As per documentation in kafka docs - >

Re: [DISCUSS] KIP-63: Unify store and downstream caching in streams

2016-06-05 Thread Eno Thereska
t and if it runs out of > memory they really struggle to fix it and kind of blame us, so I think top > down and a global config might be better. :-) > > Example: https://issues.apache.org/jira/browse/KAFKA-3775 > > -Jay > > On Fri, Jun 3, 2016 at 2:39 PM, E

[DISCUSS] KIP-63: Unify store and downstream caching in streams

2016-06-02 Thread Eno Thereska
Hi there, I have created KIP-63: Unify store and downstream caching in streams https://cwiki.apache.org/confluence/display/KAFKA/KIP-63%3A+Unify+store+and+downstream+caching+in+streams

Re: [DISCUSS] KIP-63: Unify store and downstream caching in streams

2016-06-21 Thread Eno Thereska
be subsequent KIPs where the issues around writing atomically to multiple places will be considered, and failure semantics will be strengthened. Stay tuned. Thanks Eno > > Thanks, > > Roger > > On Fri, Jun 3, 2016 at 2:39 PM, Eno Thereska <eno.there...@gmail.com> wrote:

Re: [VOTE] KIP-67: Queryable state for Kafka Streams

2016-07-08 Thread Eno Thereska
+1 (non-binding) > On 7 Jul 2016, at 18:31, Sriram Subramanian wrote: > > +1 > > On Thu, Jul 7, 2016 at 9:53 AM, Henry Cai > wrote: > >> +1 >> >> On Thu, Jul 7, 2016 at 6:48 AM, Michael Noll wrote: >> >>> +1

Re: [VOTE] KIP:71 Enable log compaction and deletion to co-exist

2016-08-15 Thread Eno Thereska
+1 (non-binding) > On 15 Aug 2016, at 14:20, Damian Guy wrote: > > I would like to initiate the voting process for KIP-71 ( > https://cwiki.apache.org/confluence/display/KAFKA/KIP-71%3A+Enable+log+compaction+and+deletion+to+co-exist > ). > > This change will add a new

Re: [DISCUSS] KIP-118: Drop Support for Java 7 in Kafka 0.11

2017-02-03 Thread Eno Thereska
Makes sense. Eno > On 3 Feb 2017, at 10:38, Ismael Juma wrote: > > Hi all, > > I have posted a KIP for dropping support for Java 7 in Kafka 0.11: > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-118%3A+Drop+Support+for+Java+7+in+Kafka+0.11 > > Most people were

Re: [DISCUSS] KIP-112: Handle disk failure for JBOD

2017-02-02 Thread Eno Thereska
of which >>> Exception >>>> should be considered disk failure and how to handle each of these are >>>> more >>>> like implementation detail. I hope we can focus on the detail and high >>>> level idea of this KIP and only worry about specif

Re: [VOTE] KIP-98: Exactly Once Delivery and Transactional Messaging

2017-02-02 Thread Eno Thereska
+1 (non-binding). Excellent work and discussions! Eno > On 2 Feb 2017, at 04:13, Guozhang Wang wrote: > > Hi all, > > We would like to start the voting process for KIP-98. The KIP can be found > at > >

Re: [DISCUSS] KIP-116 - Add State Store Checkpoint Interval Configuration

2017-02-01 Thread Eno Thereska
if it is periodic as you mention, I'm not sure that would be a show stopper. Thanks Eno > On 1 Feb 2017, at 12:58, Eno Thereska <eno.there...@gmail.com> wrote: > > Thanks Damian, this is a good idea and will reduce the restore time. Looking > forward, with exactly once and suppor

Re: [DISCUSS] KIP-112: Handle disk failure for JBOD

2017-02-01 Thread Eno Thereska
I'm coming somewhat late to the discussion, apologies for that. I'm worried about this proposal. It's moving Kafka to a world where it manages disks. So in a sense, the scope of the KIP is limited, but the direction it sets for Kafka is quite a big step change. Fundamentally this is about

Re: [DISCUSS] KIP-116 - Add State Store Checkpoint Interval Configuration

2017-02-01 Thread Eno Thereska
Thanks Damian, this is a good idea and will reduce the restore time. Looking forward, with exactly once and support for transactions in Kafka, I believe we'll have to add some support for rolling back checkpoints, e.g., when a transaction is aborted. We need to be aware of that and ideally

Re: [DISCUSS] KIP-114: KTable materialization and improved semantics

2017-02-01 Thread Eno Thereska
atthias > > > On 1/30/17 3:50 AM, Jan Filipiak wrote: >> Hi Eno, >> >> thanks for putting into different points. I want to put a few remarks >> inline. >> >> Best Jan >> >> On 30.01.2017 12:19, Eno Thereska wrote: >>> So I

Re: KIP-121 [Discuss]: Add KStream peek method

2017-02-07 Thread Eno Thereska
Hi, I like the proposal, thank you. I have found it frustrating myself not to be able to understand simple things, like how many records have been currently processed. The peek method would allow those kinds of diagnostics and debugging. Gwen, it is possible to do this with the existing

Re: [DISCUSS] KIP-112: Handle disk failure for JBOD

2017-02-07 Thread Eno Thereska
e of Kafka at the cost > of more complexity inside Kafka, by switching from RAID-10 to JBOD. I would > argue that this is a right direction. If we can gain 20%+ performance by > managing NIC in Kafka as compared to existing approach and other > alternatives, I would say we should

Re: KIP-121 [Discuss]: Add KStream peek method

2017-02-08 Thread Eno Thereska
hich is supposed to do exactly that). >>>>> >>>>> For reference, here are the descriptions of peek, map, foreach in Java >>> 8. >>>>> I could have also included links to StackOverflow questions where people >>>>> were confused abo

Re: [DISCUSS] KIP-116 - Add State Store Checkpoint Interval Configuration

2017-02-06 Thread Eno Thereska
ng on another host requires restoring from the earliest offset. > > Thanks, > Damian > > On Mon, 6 Feb 2017 at 14:58 Eno Thereska <eno.there...@gmail.com> wrote: > >> Hi Damian, >> >> I am trying to figure out if this handles a common enough failure

Re: [DISCUSS] KIP-116 - Add State Store Checkpoint Interval Configuration

2017-02-06 Thread Eno Thereska
mber-or-commit >>>> intervals". >>>> >>>> >>>> -Matthias >>>> >>>> >>>> On 2/1/17 7:29 AM, Damian Guy wrote: >>>>> Thanks for the comments Eno. >>>>> As for exactly once,

Re: [DISCUSS] KIP-112: Handle disk failure for JBOD

2017-02-01 Thread Eno Thereska
ot a complete feature. >>>> Microsoft and probably other companies are using this feature under the >>>> undesirable effect that a broker will fail any if any disk fail. It is >>> good >>>> to complete this feature. >>>> >>>> - I th

Re: [DISCUSS] KIP-114: KTable materialization and improved semantics

2017-01-30 Thread Eno Thereska
em would be implementing it > like this. > Yes I am aware that the current IQ API will not support querying by > KTableProcessorName instread of statestoreName. But I think that had to > change if you want it to be intuitive > IMO you gotta apply the filter read time > > Looking f

Re: [DISCUSS] KIP-114: KTable materialization and improved semantics

2017-01-30 Thread Eno Thereska
ns assuming we >>> get rid of the misleading function name, I personally favor not adding >> more >>> overloading functions as it keeps the API simpler. >>> >>> >>> >>> Guozhang >>> >>> >>> On Sat, Jan 28, 2017 at

Re: [DISCUSS] KIP-114: KTable materialization and improved semantics

2017-01-30 Thread Eno Thereska
n basically uses the same mechanism that is currently used. >> From my point of view this is the least confusing way for DSL users. If >> its to tricky to get a hand on the streams instance one could ask the user >> to pass it in before executing queries, therefore making sure the str

Re: [DISCUSS] KIP-114: KTable materialization and improved semantics

2017-01-28 Thread Eno Thereska
rent question is what e.g. Jan Filipiak >>>>> mentioned earlier in this thread: >>>>> I think we need to explain more clearly why KIP-114 doesn't propose the >>>>> seemingly simpler solution of always materializing tables/state stores. >>>>&

Re: [DISCUSS] KIP-114: KTable materialization and improved semantics

2017-01-28 Thread Eno Thereska
alueGetter of Filter it will apply the filter and should be completely >>> transparent as to if another processor or IQ is accessing it? How can this >>> new method help? >>> >>> I cannot see the reason for the additional materialize method being >>&g

Re: Rewind Kafka Stream consumer offset by timestamp

2017-01-29 Thread Eno Thereska
Hi Jorge, This is currently not possible, but it is likely to be considered for discussion. One challenge is that, if you have multiple topics, it is difficult to rewind them all back to a consistent point in time. KIP-95, currently under discussion, is handling the slightly different issue,

Re: [VOTE] KIP-104: Granular Sensors for Streams

2017-01-25 Thread Eno Thereska
Just FYI: between feature freeze and code freeze we realised that the name of a method didn't match its Javadoc and implementation, so changed addLatencySensor -> addLatencyAndThroughputSensor in the KIP and code. Thanks Eno > On 11 Jan 2017, at 08:10, Eno Thereska <eno.there...@

add contributor

2017-02-24 Thread Eno Thereska
Hello, Sending on behalf on user who wants to contribute. Could we add Clement Valiente to contributor list please? Jira user id is "cvaliente". Thanks Eno

Re: [DISCUSS] KIP-116 - Add State Store Checkpoint Interval Configuration

2017-02-09 Thread Eno Thereska
>>>>>>>> >>>>>>>>> Thanks Damian. >>>>>>>>> >>>>>>>>> One more question: "Checkpointing is disabled if the checkpoint >>>>>> interval >>>>>>>>> is set to a va

Re: [DISCUSS] KIP-116 - Add State Store Checkpoint Interval Configuration

2017-02-14 Thread Eno Thereska
t; On 2/10/17 9:27 AM, Damian Guy wrote: >>>> Gouzhang, Thanks for the clarification. Understood. >>>> >>>> Eno, you are correct if we just used commit interval then we wouldn't >>> need >>>> a KIP. But, then we'd have no way of turning it off.

Re: [VOTE] KIP-98: Exactly Once Delivery and Transactional Messaging

2017-02-10 Thread Eno Thereska
e the things had settled down, but it looks like that >>>> hasn't >>>>>> been done yet. I agree that it makes sense to do it before people >>> vote >>>> on >>>>>> it. >>>>>> >>>>>> Ismael >>&

Re: [DISCUSS] KIP-116 - Add State Store Checkpoint Interval Configuration

2017-02-10 Thread Eno Thereska
lue to be possibly non-positive > to indicate not turning on this feature for the reason above: if it will > not be effective then we want to leave it as an option to be turned off. > > Guozhang > > > On Fri, Feb 10, 2017 at 8:06 AM, Eno Thereska <eno.there...@gmail.com> > wrot

Re: [DISCUSS] KIP-116 - Add State Store Checkpoint Interval Configuration

2017-02-10 Thread Eno Thereska
bs. > > -dk > > On Friday 10 February 2017 04:05 PM, Eno Thereska wrote: >> I can't see why users would care to turn it off. >> >> Eno >>> On 10 Feb 2017, at 10:29, Damian Guy <damian@gmail.com> wrote: >>> >>> Hi Eno, >>&

Re: [VOTE] 0.10.2.0 RC1

2017-02-13 Thread Eno Thereska
+1 (non binding) Checked streams. Verified that stream tests work and examples off confluentinc/examples/kafka-streams work. Thanks Eno > On 10 Feb 2017, at 16:51, Ewen Cheslack-Postava wrote: > > Hello Kafka users, developers and client-developers, > > This is RC1 for

Re: [VOTE] KIP-118: Drop Support for Java 7 in Kafka 0.11

2017-02-09 Thread Eno Thereska
+1 (non binding) > On 9 Feb 2017, at 15:53, Damian Guy wrote: > > +1 > > On Thu, 9 Feb 2017 at 15:41 Bill Bejeck wrote: > >> +1 >> >> On Thu, Feb 9, 2017 at 10:31 AM, Ismael Juma wrote: >> >>> Hi everyone, >>> >>> Since everyone

Re: KIP-121 [VOTE]: Add KStream peek method

2017-02-09 Thread Eno Thereska
+1 Thanks Eno > On 9 Feb 2017, at 11:04, Damian Guy wrote: > > +1 > > On Thu, 9 Feb 2017 at 10:26 Michael Noll wrote: > >> +1 (non-binding) >> >> And thanks again for suggesting and driving this, Steven! :-) >> >> Best, >> Michael >> >> >> >>

Re: [DISCUSS] KIP-116 - Add State Store Checkpoint Interval Configuration

2017-02-10 Thread Eno Thereska
as really high), about allowing it to be "turned off" >> with a >>>> non-positive value: I feel there are still values to keep this door >> open as >>>> in the future if EOS is turned on, people may just want to turn off >>>> checkpoint

Re: [DISCUSS] KIP-116 - Add State Store Checkpoint Interval Configuration

2017-02-10 Thread Eno Thereska
values to keep this door open as >> in the future if EOS is turned on, people may just want to turn off >> checkpointing anyways, or there maybe other scenarios that we have not >> realized yet. On the other hand, I would argue that it is less likely users >> mistakenly se

Re: [VOTE] 0.10.2.0 RC0

2017-02-09 Thread Eno Thereska
Unfortunately we had a regression in streams and KAFKA-4716 is now deemed a Blocker. https://issues.apache.org/jira/browse/KAFKA-4716 Eno > On 7 Feb 2017, at 23:56, Jun Rao wrote: > > Hi, Ewen, > > KAFKA-4725 has been

Re: KIP-121 [VOTE]: Add KStream peek method

2017-02-15 Thread Eno Thereska
KIP is accepted, discussion now moves to PR. Thanks Eno On Wed, Feb 15, 2017 at 12:28 PM, Steven Schlansker < sschlans...@opentable.com> wrote: > Oops, sorry, a number of votes were sent only to -dev and not to > -user and so I missed those in the email I just sent. The actual count is > more

Re: [DISCUSS] KIP-114: KTable materialization and improved semantics

2017-01-18 Thread Eno Thereska
KIP > proposes to remove existing API methods (such as `KTable#foreach()`). > > -Michael > > > On Tue, Jan 17, 2017 at 10:55 AM, Eno Thereska <eno.there...@gmail.com> > wrote: > >> Thanks Damian, answers inline: >> >>> On 16 Jan 2017, at 17:17, Da

Re: [DISCUSS] KIP-114: KTable materialization and improved semantics

2017-01-18 Thread Eno Thereska
t; Rename toStream() to toKStream() for consistency. > > Not sure whether that is really required. We also use > `KStreamBuilder#stream()` and `KStreamBuilder#table()`, for example, and > don't care about the "K" prefix. > > > > On Tue, Jan 17, 2017 at 10:55 AM, Eno The

Re: [DISCUSS] KIP-114: KTable materialization and improved semantics

2017-01-18 Thread Eno Thereska
embodiment of a Kafka changelog topic. It comes with a set of unique APIs too. Thanks Eno > > > > > > On Tue, Jan 17, 2017 at 10:55 AM, Eno Thereska <eno.there...@gmail.com> > wrote: > >> Thanks Damian, answers inline: >> >>> On 16 Jan 2017,

Re: [DISCUSS] KIP-114: KTable materialization and improved semantics

2017-01-16 Thread Eno Thereska
ts own > KIP as interested parties may miss it if it's inside a Streams KIP. It > would also be good to elaborate on the details of how this new log > compaction mode would work. > > Ismael > > On Mon, Jan 16, 2017 at 4:44 PM, Eno Thereska <eno.there...@gmail.com

Re: [DISCUSS] KIP-114: KTable materialization and improved semantics

2017-01-17 Thread Eno Thereska
KTable Join operators will also need Serde > information. I'll update the KIP with the serdes. Thanks Eno > > Cheers, > Damian > > > On Mon, 16 Jan 2017 at 16:44 Eno Thereska <eno.there...@gmail.com> wrote: > >> Hello, >> >> We created &qu

[DISCUSS] KIP-114: KTable materialization and improved semantics

2017-01-16 Thread Eno Thereska
Hello, We created "KIP-114: KTable materialization and improved semantics" to solidify the KTable semantics in Kafka Streams: https://cwiki.apache.org/confluence/display/KAFKA/KIP-114%3A+KTable+materialization+and+improved+semantics

Re: [DISCUSS] KIP-112: Handle disk failure for JBOD

2017-02-28 Thread Eno Thereska
Makes sense, thank you Dong. Eno > On 28 Feb 2017, at 01:51, Dong Lin wrote: > > Hi Jun, > > In addition to the Eno's reference of why rebuild time with RAID-5 is more > expensive, another concern is that RAID-5 will fail if more than one disk > fails. JBOD is still works

Re: [DISCUSS] KIP-123: Allow per stream/table timestamp extractor

2017-02-28 Thread Eno Thereska
Hi Jeyhun, Thanks for the KIP, sorry I'm coming a bit late to the discussion. One thing I'd like to understand is whether we can avoid situations where the user is mixing different times (event time vs. wallclock time) in their processing inadvertently. Before this KIP, all the relevant topics

Re: [DISCUSS] KIP-123: Allow per stream/table timestamp extractor

2017-02-28 Thread Eno Thereska
gt; > > Cheers, > Jeyhun > > On Tue, Feb 28, 2017 at 10:41 AM Eno Thereska <eno.there...@gmail.com> > wrote: > >> Hi Jeyhun, >> >> Thanks for the KIP, sorry I'm coming a bit late to the discussion. >> >> One thing I'd like to understand i

Re: [VOTE] KIP-123: Allow per stream/table timestamp extractor

2017-02-28 Thread Eno Thereska
That's ok, that was my fault for looking at this thread late. We can leave the vote thread as is and simultaneously answer any remaining questions on the discuss thread. Thanks Eno > On 28 Feb 2017, at 12:55, Michael Noll wrote: > > Thanks, Jeyhun! > > I observed that

Re: [DISCUSS] KIP-112: Handle disk failure for JBOD

2017-02-27 Thread Eno Thereska
; rebuild? RAID-10 seems to still be used widely. > > Jun > > On Mon, Feb 27, 2017 at 1:38 PM, Eno Thereska <eno.there...@gmail.com> > wrote: > >> Unfortunately RAID-5/6 is not typically advised anymore due to failure >> issues, as Dong mentions, e.g.: htt

Re: [DISCUSS] KIP-112: Handle disk failure for JBOD

2017-02-27 Thread Eno Thereska
Unfortunately RAID-5/6 is not typically advised anymore due to failure issues, as Dong mentions, e.g.: http://www.zdnet.com/article/why-raid-6-stops-working-in-2019/ Eno > On 27 Feb 2017, at 21:16, Jun Rao

Re: [DISCUSS] KIP-112: Handle disk failure for JBOD

2017-02-28 Thread Eno Thereska
Thanks Todd for the explanation. Eno > On 28 Feb 2017, at 18:15, Todd Palino wrote: > > We have tested RAID 5/6 in the past (and recently) and found it to be > lacking. So, as noted, rebuild takes more time than RAID 10 because all the > disks need to be accessed to

[VOTE] KIP-63: Unify store and downstream caching in streams

2016-08-25 Thread Eno Thereska
Hi folks, We'd like to start the vote for KIP-63. At this point the Wiki addresses all previous questions and we believe the PoC is feature-complete. Thanks Eno

Re: [DISCUSS] KIP-63: Unify store and downstream caching in streams

2016-09-05 Thread Eno Thereska
> -Matthias > > > On 09/04/2016 07:10 PM, Matthias J. Sax wrote: >> Sure, you can use a non-cached state. However, if you write code like >> below for a non-cached state, and learn about caching later on, and >> think, caching is a cool feature, I want to use it, you would s

Re: [DISCUSS] KIP-63: Unify store and downstream caching in streams

2016-09-06 Thread Eno Thereska
A small update to the KIP: the deduping of records using the cache does not affect the .to operator since we'd have already deduped the KTable before the operator. Adjusting KIP. Thanks Eno > On 5 Sep 2016, at 12:43, Eno Thereska <eno.there...@gmail.com> wrote: > &

Re: [ANNOUNCE] New committer: Jason Gustafson

2016-09-07 Thread Eno Thereska
Congrats Jason! Eno > On 7 Sep 2016, at 10:00, Rajini Sivaram wrote: > > Congrats, Jason! > > On Wed, Sep 7, 2016 at 8:29 AM, Flavio P JUNQUEIRA wrote: > >> Congrats, Jason. Well done and great to see this project inviting new >> committers. >>

Re: [DISCUSS] KIP-63: Unify store and downstream caching in streams

2016-09-04 Thread Eno Thereska
Hi Matthias, Thanks for the good questions. There is still the option of not using cached state. If one uses cached state it will dedup for stores and forwarding further. But you can always disable caching and do what you say. Eno > On 4 Sep 2016, at 17:36, Matthias J. Sax

Re: [VOTE] KIP-63: Unify store and downstream caching in streams

2016-09-08 Thread Eno Thereska
ski <j...@jagunet.com> wrote: > > +1 >> On Aug 25, 2016, at 6:57 AM, Eno Thereska <eno.there...@gmail.com> wrote: >> >> Hi folks, >> >> We'd like to start the vote for KIP-63. At this point the Wiki addresses >> all previous questions and we believe the PoC is feature-complete. >> >> Thanks >> Eno >

Re: [VOTE] KIP-63: Unify store and downstream caching in streams

2016-09-08 Thread Eno Thereska
> state store engines (i.e. RocksDB), and instead add a per-thread global > cache which will only be activated for state stores used in Streams DSL's > aggregate operator as a write deduplicator for both the state store and to > the downstream operators"? > > Guozhang >

Re: [DISCUSS] KIP-63: Unify store and downstream caching in streams

2016-09-08 Thread Eno Thereska
nd is decoupled with its logic of forwarding / storing in persistent >> state stores. >> >> One may argue that now if users want to make use of the cache, he will need >> to make code changes; but I think this is a reasonable requirement to users >> actually, since that 1

Re: [VOTE] KIP-63: Unify store and downstream caching in streams

2016-09-12 Thread Eno Thereska
Forgot to include the stats: There were 3 binding votes (Guozhang, Neha, Gwen) and 6 non-binding votes (Matthias, Bill, Michael, Jim, Damian, Eno). Thanks to everyone that voted and participated in the discussions. Eno > On 12 Sep 2016, at 16:00, Eno Thereska <eno.there...@gmail.com&

Re: [VOTE] KIP-63: Unify store and downstream caching in streams

2016-09-12 Thread Eno Thereska
nd very detailed. Nice job, Eno :) >> >> On Thu, Aug 25, 2016 at 3:57 AM, Eno Thereska <eno.there...@gmail.com> >> wrote: >> >>> Hi folks, >>> >>> We'd like to start the vote for KIP-63. At this point the Wiki addresses >>> all prev

Re: Queryable state client read guarantees

2016-08-27 Thread Eno Thereska
ter/kafka-streams/src/main/java/io/confluent/examples/streams/queryablestate>. The instructions on how to run are in QueryableStateExample.java. Thanks Eno > On 26 Aug 2016, at 18:07, Eno Thereska <eno.there...@gmail.com> wrote: > > Hi Mikael, > > Very good ques

Re: Queryable state client read guarantees

2016-08-26 Thread Eno Thereska
Hi Mikael, Very good question. You are correct about the desired semantics. The semantic of case (a) depends on the local store as you mention. For case (b), the final check is always performed again on get(), and if the store has disappeared between the lookup and get, the user will get an

Re: [DISCUSS] KIP-63: Unify store and downstream caching in streams

2016-08-24 Thread Eno Thereska
for disabling caching on a store-by-store basis using a new .enableCaching() call. We'll start the voting process shortly for this KIP. Thanks Eno On Thu, Jun 2, 2016 at 11:30 AM, Eno Thereska <eno.there...@gmail.com> wrote: > Hi there, > > I have created KIP-63: Unify store and dow

Re: [VOTE] KIP-77: Improve Kafka Streams Join Semantics

2016-08-29 Thread Eno Thereska
+1 (non-binding) > On 29 Aug 2016, at 12:22, Bill Bejeck wrote: > > +1 > > On Mon, Aug 29, 2016 at 5:50 AM, Matthias J. Sax > wrote: > >> I’d like to initiate the voting process for KIP-77: >> >>

Re: [ANNOUNCE] New committer: Jiangjie (Becket) Qin

2016-11-02 Thread Eno Thereska
Congrats! Eno > On 1 Nov 2016, at 05:57, Harsha Chintalapani wrote: > > Congrats Becket! > -Harsha > > On Mon, Oct 31, 2016 at 2:13 PM Rajini Sivaram > wrote: > >> Congratulations, Becket! >> >> On Mon, Oct 31, 2016 at 8:38 PM, Matthias J. Sax

Re: Store flushing on commit.interval.ms from KIP-63 introduces aggregation latency

2016-10-11 Thread Eno Thereska
t; system without increasing latency, but maybe I'm missing something. > > > On Oct 10, 2016 3:10 AM, "Eno Thereska" <eno.there...@gmail.com> wrote: > >> Hi Greg, >> >> Thanks for trying 0.10.1. The best option you have for your spec

Re: Store flushing on commit.interval.ms from KIP-63 introduces aggregation latency

2016-10-10 Thread Eno Thereska
Hi Greg, Thanks for trying 0.10.1. The best option you have for your specific app is to simply turn off caching by setting the cache size to 0. That should give you the old behaviour: streamsConfiguration.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0L); Your PR is an alternative, but

Re: [VOTE] 0.10.1.0 RC3

2016-10-15 Thread Eno Thereska
+1 (non-binding) > On 15 Oct 2016, at 17:47, Jason Gustafson wrote: > > As promised, here are updated test results: > > Unit tests: https://builds.apache.org/job/kafka-0.10.1-jdk7/75/ > System tests: >

Re: [VOTE] KIP-94: Session Windows

2016-12-06 Thread Eno Thereska
+1 (non-binding) Thanks Eno > On 6 Dec 2016, at 12:09, Damian Guy wrote: > > Hi all, > > I'd like to start the vote for KIP-94: > https://cwiki.apache.org/confluence/display/KAFKA/KIP-94+Session+Windows > > There is a PR for it here:

Re: [VOTE] KIP-100 - Relax Type constraints in Kafka Streams API

2016-12-12 Thread Eno Thereska
+1 (non-binding) > On 12 Dec 2016, at 15:35, Bill Bejeck wrote: > > +1 > > On Mon, Dec 12, 2016 at 3:44 AM, Damian Guy wrote: > >> +1 >> >> On Sun, 11 Dec 2016 at 08:07 Ewen Cheslack-Postava >> wrote: >> >>> +1 (binding) >>> >>>

Re: [DISCUSS] KIP-104: Granular Sensors for Streams

2017-01-09 Thread Eno Thereska
eter to just `Sensor` > instead of `String`? > > > Guozhang > > > > > > On Mon, Jan 9, 2017 at 10:13 AM, Eno Thereska <eno.there...@gmail.com> > wrote: > >> One hopefully final update to the KIP to re-add two methods for generic >> sensor

Re: [DISCUSS] KIP-105: Addition of Record Level for Sensors

2017-01-09 Thread Eno Thereska
t; information is something that can be done later, as you said. >> >> Ismael >> >> On Fri, Jan 6, 2017 at 10:24 AM, Eno Thereska <eno.there...@gmail.com> >> wrote: >> >>> To clarify an earlier point Ismael made, MetricReporter implementations >>> h

Re: [VOTE] KIP-104: Granular Sensors for Streams

2017-01-11 Thread Eno Thereska
1:05 PM Guozhang Wang <wangg...@gmail.com> wrote: > >> +1, thanks. >> >> On Jan 9, 2017 12:27 PM, "Ismael Juma" <ism...@juma.me.uk> wrote: >> >> Thanks for the KIP, +1 on the latest update. >> >> Ismael >> >> On Fri,

Re: [ANNOUNCE] New committer: Grant Henke

2017-01-11 Thread Eno Thereska
Congrats! Eno > On 11 Jan 2017, at 20:06, Ben Stopford wrote: > > Congrats Grant!! > On Wed, 11 Jan 2017 at 20:01, Ismael Juma wrote: > >> Congratulations Grant, well deserved. :) >> >> Ismael >> >> On 11 Jan 2017 7:51 pm, "Gwen Shapira"

Re: [DISCUSS] KIP-95: Incremental Batch Processing for Kafka Streams

2016-11-30 Thread Eno Thereska
helps batch-mode streams apps to be good-citizens when running in shared > environments. > > Thanks, > Damian > > On Wed, 30 Nov 2016 at 10:40 Eno Thereska <eno.there...@gmail.com> wrote: > >> Hi Matthias, >> >> I like the first part of the KIP. Howe

Re: [DISCUSS] KIP-95: Incremental Batch Processing for Kafka Streams

2016-11-30 Thread Eno Thereska
ng else > is still completely decoupled. The only add-on if for consumer of > intermediate topics -- they don't pick up the stop offsets at start up > but whenever they are available later on. > > > Does this make sense? > > > -Matthias > > On 11/30/16 2:40

Re: [VOTE] KIP-96 - Add per partition metrics for in-sync and replica count

2016-11-30 Thread Eno Thereska
+1 (non binding) > On 30 Nov 2016, at 21:34, Xavier Léauté wrote: > > Based on the feedback KIP-96 seems pretty uncontroversial, so I'd like to > initiate a vote on it. > >

Re: [DISCUSS] KIP-95: Incremental Batch Processing for Kafka Streams

2016-11-30 Thread Eno Thereska
Hi Matthias, I like the first part of the KIP. However, the second part with the failure modes and metadata topic is quite complex and I'm worried it doesn't solve the problems you mention under failure. For example, the application can fail before writing to the metadata topic. In that case,

Re: [DISCUSS] KIP-95: Incremental Batch Processing for Kafka Streams

2016-12-04 Thread Eno Thereska
A couple of remaining questions: - it says in the KIP: "because the metadata topic must be consumed by all instances, we need to assign the topic’s partitions manually and do not commit offsets -- we also need to seekToBeginning() each time we consume the metadata topic)" . How big of a change

Re: Suppressing redundant KTable forwards

2016-12-04 Thread Eno Thereska
Hi Mathieu, Thanks for the suggestion. Wouldn't the cache introduced in KIP-63 do some of this for you, in that it dedups records with the same key and prevents them from being forwarded downstream? Eno > On 4 Dec 2016, at 04:13, Mathieu Fenniak wrote: > > Hey

[VOTE] KIP-104: Granular Sensors for Streams

2017-01-06 Thread Eno Thereska
The discussion points for KIP-104 are addressed. At this point we'd like to start the vote for it: https://cwiki.apache.org/confluence/display/KAFKA/KIP-104%3A+Granular+Sensors+for+Streams Thanks, Eno

Re: [DISCUSS] KIP-105: Addition of Record Level for Sensors

2017-01-05 Thread Eno Thereska
he PR as >> well regarding this idea. My concern is that it may be not very >> straight-forward to implement though via the MetricsReporter interface, if >> Eno and Aarti has a good approach to tackle it I would love it. >> >> >> Guozhang >> >> On

Re: [DISCUSS] KIP-105: Addition of Record Level for Sensors

2017-01-05 Thread Eno Thereska
I would love it. > > > Guozhang > > On Thu, Jan 5, 2017 at 5:34 AM, Eno Thereska <eno.there...@gmail.com> wrote: > >> Updated KIP for 1. Waiting to hear from Guozhang on 2 and then we can >> proceed. >> >> Thanks >> Eno >>> On 5

Re: [DISCUSS] KIP-104: Granular Sensors for Streams

2017-01-06 Thread Eno Thereska
So the KIP proposes exposing the metrics registry (second paragraph under motivation). The community has indicated that they would like to 1. access all the metrics and 2. register their own. We provide some helper interfaces for them to register throughput and latency metrics, but ultimately

Re: [DISCUSS] KIP-104: Granular Sensors for Streams

2017-01-06 Thread Eno Thereska
t; not >>> currently exposed by anything else. Traditionally, we list all public >> APIs >>> created by a KIP, which is effectively true for the registry in this >> case. >>> Did we consider using an interface instead of the concrete class? It >> seems

Re: [VOTE] KIP-104: Granular Sensors for Streams

2017-01-06 Thread Eno Thereska
: >>> >>>> +1 >>>> >>>> On Fri, Jan 6, 2017 at 5:57 AM, Damian Guy <damian@gmail.com> >> wrote: >>>> >>>>> +1 >>>>> >>>>> On Fri, 6 Jan 2017 at 09:37 Eno Thereska <eno.there.

Re: [DISCUSS] KIP-104: Granular Sensors for Streams

2017-01-06 Thread Eno Thereska
ace instead of the concrete class? It seems > that a lot of these things were discussed in the PR, so it would be good to > have a summary in the KIP too. > > Ismael > > On Fri, Jan 6, 2017 at 9:10 PM, Eno Thereska <eno.there...@gmail.com> wrote: > >> So the KIP pr

Re: [DISCUSS] KIP-105: Addition of Record Level for Sensors

2017-01-05 Thread Eno Thereska
eporter but not recorded. To an end-user's experience it will mean that >> for example the monitoring UI that displays all polled metrics will still >> show the metrics graph, with the value consistently shown as the default >> value, instead of not showing the graphs at all. >

Re: [DISCUSS] KIP-105: Addition of Record Level for Sensors

2017-01-05 Thread Eno Thereska
ing recorded). > > Ismael > > On Thu, Jan 5, 2017 at 10:37 AM, Eno Thereska <eno.there...@gmail.com> > wrote: > >> Correct on 2. Guozhang: the sensor will be registered and polled by a >> reporter, but the metrics associated with it will not be updated. >>

Re: [DISCUSS] KIP-105: Addition of Record Level for Sensors

2017-01-05 Thread Eno Thereska
g a field could break existing tools. > Having said that, your suggestion not to register sensors at all if their > record level is below what is configured works for me. > > Ismael > > On Thu, Jan 5, 2017 at 12:07 PM, Eno Thereska <eno.there...@gmail.com> > wrote: >

Re: [DISCUSS] KIP-104: Granular Sensors for Streams

2017-01-09 Thread Eno Thereska
hat's the name > we use in the consumer and producer instead of `readOnlyRegistry`?) is fine > by me as the conservative option. > > If we think that access to the full registry is really important and others > are fine with the current `Metrics` API, I won't stand in the way though. > &

Re: [DISCUSS] KIP-104: Granular Sensors for Streams

2017-01-09 Thread Eno Thereska
conservative than what was before. Eno > On 9 Jan 2017, at 10:31, Eno Thereska <eno.there...@gmail.com> wrote: > > Thanks Ismael, Damian, I have updated KIP to go with the conservative option. > > Eno >> On 9 Jan 2017, at 09:44, Ismael Juma <ism...@juma.m

Re: [DISCUSS] KIP-105: Addition of Record Level for Sensors

2017-01-01 Thread Eno Thereska
Thanks for starting the discussion on these KIPs Aarti. Eno On Sunday, January 1, 2017, Aarti Gupta wrote: > Thanks Radai, > > Yes that is the correct link, my bad > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP- > 105%3A+Addition+of+Record+Level+for+Sensors

Re: [DISCUSS] KIP-104: Granular Sensors for Streams

2017-01-08 Thread Eno Thereska
asurable measurable); > public synchronized KafkaMetric removeMetric(MetricName metricName); > > public synchronized void addReporter(MetricsReporter reporter); > > public Map<MetricName, KafkaMetric> metrics(); > > public KafkaMetric metric(MetricName metricName); > > O

Re: [DISCUSS] KIP-104: Granular Sensors for Streams

2017-01-08 Thread Eno Thereska
r vote on the previous approach or add a note to this discuss thread please since we're getting close to the feature freeze deadline. Thanks Eno > On 8 Jan 2017, at 15:06, Eno Thereska <eno.there...@gmail.com> wrote: > > Ismael, > > Based on the streams user demand for registering t

  1   2   3   4   5   6   7   >