Hi Bill,
Thanks for the KIP. Now that we're using strings describing the "set of
optimizations", such as "none" and "all", should we change the config name
to just "topology.optimizations"?
The "enable." feels like a holdover from the boolean-valued config.
Thanks,
-John
On Tue, May 8, 2018 at
Thanks for the KIP, Guozhang.
It looks good overall to me; I just have one question:
* Why do we bound the generics of KVMapper in KStream to be superclass-of-K
and superclass-of-V instead of exactly K and V, as in Topology? I might be
thinking about it wrong, but that seems backwards to me. If
+1 non-binding
On Thu, May 17, 2018 at 4:44 PM, Matthias J. Sax
wrote:
> +1 (binding)
>
>
> On 5/17/18 12:18 PM, Ted Yu wrote:
> > +1
> > Original message From: Gwen Shapira
> Date: 5/17/18 11:53 AM (GMT-08:00) To: dev
Hi Bill,
Thanks for the KIP.
Just a small thought. This new API will result in calls that look like this:
new KafkaStreams(builder.build(props), props);
Do you think that's a significant enough eyesore to warrant adding a new
KafkaStreams constructor taking a KStreamsBuilder like this:
new
>> working PR we're reusing InternalStreamsBuilder for the logical plan
> >> generation) which can then be called inside KafkaStreams constructors?
> >>
> >>
> >> Guozhang
> >>
> >>
> >> On Mon, Jun 11, 2018 at 9:41 AM, John Roesler
> wrote:
Hello devs and users,
Please take some time to consider this proposal for Kafka Streams:
KIP-328: Ability to suppress updates for KTables
link: https://cwiki.apache.org/confluence/x/sQU0BQ
The basic idea is to provide:
* more usable control over update rate (vs the current state store caches)
lly the same as what I
> suggested.
> > So it is good by me.
> >
> > Thanks
> >
> > On Wed, Jun 27, 2018 at 7:31 AM, John Roesler wrote:
> >
> >> Thanks for taking look, Ted,
> >>
> >> I agree this is a departure from the conve
Hello again all,
I realized today that I neglected to include metrics in the proposal. I
have added them just now.
Thanks,
-John
On Tue, Jun 26, 2018 at 3:11 PM John Roesler wrote:
> Hello devs and users,
>
> Please take some time to consider this proposal for Kafka Streams:
&g
ructures by supporting `of` ?
>
> Suppression.of(Duration.ofMinutes(10))
>
>
> Cheers
>
>
>
> On Tue, Jun 26, 2018 at 1:11 PM, John Roesler wrote:
>
> > Hello devs and users,
> >
> > Please take some time to consider this proposal for Kafka
estion may be to implementation specific but if the requested
> suppression time is longer than the specified commit time, will the latest
> record in the suppression buffer get stored in a changelog?
>
> Thanks,
> Bill
>
> On Wed, Jun 27, 2018 at 3:04 PM John Roesler wrote:
> >
> > Thanks,
> > Bill
> >
> > On Mon, Jun 25, 2018 at 4:17 PM John Roesler wrote:
> >
> >> Thanks Matthias and Guozhang,
> >>
> >> About deprecating the "segments" field instead of making it private.
> Yes, I
> >> j
Hello All,
Thanks for the discussion on KIP-319. I'd now like to start the voting.
As a reminder, KIP-319 proposes a fix to an issue I identified in
KAFKA-7080. Specifically, the issue is that we're creating
CachingWindowStore with the *number of segments* instead of the *segment
size*.
Here's
ameter but an implementation detail and thus a store
> > parameter. Would it be better to add it to `Materialized`?
> >
> >
> > -Matthias
> >
> > On 6/22/18 5:13 PM, Guozhang Wang wrote:
> > > Thanks John.
> > >
> > > On Fri, Jun 22, 20
Ah, it turns out I did create a ticket: it's KAFKA-7080:
https://issues.apache.org/jira/browse/KAFKA-7080
-John
On Mon, Jun 25, 2018 at 4:44 PM John Roesler wrote:
> Matthias,
>
> That's a good idea. I'm not sure why I didn't...
>
> Thanks,
> -John
>
> On Mon, Jun 25,
d name is still segmentSize
> in
> > the code block vs segmentInterval and the order of the parameters for the
> > third persistentWindowStore don't match the order in the JavaDoc.
> >
> > Thanks,
> > Bill
> >
> >
> >
> > On Thu, Jun 21, 2018 at 3
Hello All,
I'd like to propose KIP-319 to fix an issue I identified in KAFKA-7080.
Specifically, we're creating CachingWindowStore with the *number of
segments* instead of the *segment size*.
Here's the jira: https://issues.apache.org/jira/browse/KAFKA-7080
Here's the KIP:
segmentInterval ?
> Thanks
> Original message From: John Roesler
> Date: 6/20/18 10:45 AM (GMT-08:00) To: dev@kafka.apache.org Subject:
> [DISCUSS] KIP-319: Replace segments with segmentSize in
> WindowBytesStoreSupplier
> Hello All,
>
> I'd like to prop
Hello All,
I'd like to propose KIP-318 to fix an issue I identified in KAFKA-7080.
Specifically, we're creating CachingWindowStore with the *number of
segments* instead of the *segment size*.
Here's the jira: https://issues.apache.org/jira/browse/KAFKA-7080
Here's the KIP:
Oops! It looks like 318 was taken. I'll re-send this message to a new
thread.
On Wed, Jun 20, 2018 at 12:40 PM John Roesler wrote:
> Hello All,
>
> I'd like to propose KIP-318 to fix an issue I identified in KAFKA-7080.
> Specifically, we're creating CachingWindowStore with
Sorry for the late comment,
Looking at the other pieces of TopologyDescription, I noticed that pretty
much all of the "payload" of these description nodes are strings. Should we
consider returning a string from `topicNameExtractor()` instead?
In fact, if we did that, we could consider calling
; I meant to create a JIRA to add `segmentInterval` to `Materialized` and
> a JIRA to add `Materialized` to `KStream#join(KStream)`.
>
> Thx.
>
>
> -Matthias
>
> On 6/25/18 2:46 PM, John Roesler wrote:
> > Ah, it turns out I did create a ticket: it's KAFKA-7080:
&g
Could you explain better? If that
> is possible I think it would be great.
>
> Thanks for the intervention!
>
> -Flávio Stutz
>
>
>
>
> On 2018/07/02 20:03:57, John Roesler wrote:
> > Hi Flávio,
> >
> > Thanks for the KIP. I'll apologize that I'm a
Hi Flávio,
Thanks for the KIP. I'll apologize that I'm arriving late to the
discussion. I've tried to catch up, but I might have missed some nuances.
Regarding KIP-328, the idea is to add the ability to suppress intermediate
results from all KTables, not just windowed ones. I think this could
both expressed doubt that there
are practical use cases for it outside of final-results.
-John
On Mon, Jul 2, 2018 at 12:27 PM John Roesler wrote:
> Hi again, Guozhang ;) Here's the second part of my response...
>
> It seems like your main concern is: "if I'm a user who wants final u
oes "suppressLateEvents" with parameter Y != X (window retention time)
> for windowed stores make sense in practice?
> 2. Does "suppressLateEvents" with any parameter Y for non-windowed stores
> make sense in practice?
>
>
>
> Guozhang
>
>
&
Hi Chia-Ping,
I couldn't find KIP-331 in the list of KIPs (
https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals
).
Can you provide a link, please?
Thanks,
-John
On Sun, Jul 1, 2018 at 11:33 AM Chia-Ping Tsai wrote:
> hi folks,
>
> KIP-331 is waiting for any
ot; with any parameter Y for non-windowed stores
> make sense in practice?
>
>
>
> Guozhang
>
>
> On Fri, Jun 29, 2018 at 2:26 PM, Bill Bejeck wrote:
>
> > Thanks for the explanation, that does make sense. I have some questions
> on
> > operations, but
>
> To reply your email on "Mon, Jul 2, 2018 at 10:27 AM":
>
> I'd like option 2) over option 1) better as well from programming pov. But
> I'm wondering if option 2) would provide the above semantics or i
Hi Jorge,
Thanks for the design work.
I agree that de-scoping the work to just the Processor API will help
contain the design and implementation complexity.
In the KIP, it mentions that the headers would be available in the
ProcessorContext, (like "context.headers()"). It also says that
a single one to control timeout
> > semantics it may be even confusing: take our producer side configs for an
> > example, right now we have "request.timeout.ms" and "max.block.ms" and
> we
> > are proposing to add another one in KIP-91. But I'd also like to h
orth.
>
> -Jason
>
>
> On Wed, May 2, 2018 at 3:26 PM, Richard Yu <yohan.richard...@gmail.com>
> wrote:
>
> > Hi John,
> >
> > I don't have any objections to this KIP change. Please go ahead.
> >
> > Thanks,
> > Richard
> >
> > O
context.
> I think using interval in the method name would clearly convey the meaning
> intuitively.
>
> Thanks
>
>
> On Wed, Jun 20, 2018 at 1:31 PM, John Roesler wrote:
>
> > Hi Ted,
> >
> > Ah, when you made that comment to me before, I thought you me
I've updated the KIP and draft PR accordingly.
On Thu, Jun 21, 2018 at 2:03 PM John Roesler wrote:
> Interesting... I did not initially consider it because I didn't want to
> have an impact on anyone's Streams apps, but now I see that unless
> developers have subclassed `Windows`, t
e KIP.
>
> Should we consider making the change on `Stores#persistentWindowStore`
> parameters as well?
>
>
> Guozhang
>
>
> On Wed, Jun 20, 2018 at 1:31 PM, John Roesler wrote:
>
> > Hi Ted,
> >
> > Ah, when you made that comment to me before, I though
Hello devs,
The discussion of KIP-328 has gone some time with no new comments, so I am
calling for a vote!
Here's the KIP: https://cwiki.apache.org/confluence/x/sQU0BQ
The basic idea is to provide:
* more usable control over update rate (vs the current state store caches)
* the
ion operators from the `skipped-records` metrics recording to the
> `late-event-drop` metrics recording.
>
>
>
> Guozhang
>
>
> On Mon, Jul 30, 2018 at 1:36 PM, Bill Bejeck wrote:
>
> > Thanks for the KIP!
> >
> > +1
> >
> >
ordingly.
Hopefully, this doesn't change anyone's vote.
Thanks,
-John
On Mon, Jul 30, 2018 at 5:31 PM John Roesler wrote:
> Thanks Guozhang,
>
> Thanks for that catch. to clarify, currently, events are "late" only when
> they are older than the retention period. Currently, we
Hi Boyang,
Overall, this seems like a good addition to the consumer.
I agree with the others that we should attempt to validate the uniqueness
of member.id usage. FWIW, Jason's idea of using a configured logical id +
assigned unique id seems to be suitably flexible and free of assumptions,
as
id generation, which brought up multiple validation
> > concerns. As the host:port approach is vetoed, and I have thought for a
> > while for other validation strategies but failed, I think it's time to
> > decide whether we want to focus our next step discussion on
> >
> >
&
Hi Cédric,
The suffix is generated when we build the topology in such a way to
guarantee each node/interna-topic/state-store gets a unique name.
Generally speaking, it is unsafe to modify the topology and restart it. We
recommend using the app reset tool whenever you update your topology.
That
-John
On Tue, Aug 7, 2018 at 12:07 PM John Roesler wrote:
> Thanks everyone, KIP-328 has passed with 3 binding votes (Guozhang,
> Damian, and Matthias) and 3 non-binding (Ted, Bill, and me).
>
> Thanks for your time,
> -John
>
> On Mon, Aug 6, 2018 at 6:35 PM Matthias J. Sax
> On 8/3/18 12:52 AM, Damian Guy wrote:
> > Thanks John! +1
> >
> > On Mon, 30 Jul 2018 at 23:58 Guozhang Wang wrote:
> >
> >> Yes, the addendum lgtm as well. Thanks!
> >>
> >> On Mon, Jul 30, 2018 at 3:34 PM, John Roesler
> wrote:
>
I also have no comments. The KIP looks good to me.
-John
On Thu, Aug 9, 2018 at 1:26 PM Matthias J. Sax
wrote:
> @Guozhang, I think you can start the VOTE for this KIP? I don't have any
> further comments.
>
> One more nit: we should explicitly state, that the new config is
> wall-clock time
; KIP-307: Allow to define custom processor names with KStreams DSL
> > <https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > 307%3A+Allow+to+define+custom+processor+names+with+KStreams+DSL>
> >
> > I know that the probalilty a KEY-SELECT node get the same number
t;>>>
> > >>>>>>>> (Same for other open questions like interface names -- I collect
> > >>>>>>>> feedback and update the KIP after we reach consensus :))
> > >>>>>>>>
> > >>>>>
+1 non-binding
On Thu, Aug 9, 2018 at 3:14 PM Matthias J. Sax
wrote:
> +1 (binding)
>
> On 8/9/18 11:57 AM, Guozhang Wang wrote:
> > Hello all,
> >
> > I would like to start the voting processing on the following KIP, to
> allow
> > users control when a task can be processed based on its
This sounds good to me!
Thanks for the time you've spent on it,
-John
On Tue, Aug 21, 2018 at 12:13 AM Boyang Chen wrote:
> Thanks Matthias for the input. Sorry I was busy recently and haven't got
> time to update this thread. To summarize what we come up so far, here is a
> draft updated
Quick afterthought: I guess that `Window` is exposed to the API via
`Windowed` keys. I think it would be fine to not deprecate the `long` start
and end, but add `Instant` variants for people preferring that interface.
On Fri, Aug 24, 2018 at 11:10 AM John Roesler wrote:
> Hey Matth
, it might make sense to keep old and just add new ones? End users
> > can use the "nicer" new ones, while we can still use the existing ones
> > internally? Not sure if it would be possible to keep the old ones
> > without exposing them as public API?
> >
> >
I'm also in favor of this. I don't think it's controversial either. Should
we just move to a vote?
On Thu, Aug 23, 2018 at 7:01 PM Guozhang Wang wrote:
> +1.
>
> On Thu, Aug 23, 2018 at 12:47 PM, Ted Yu wrote:
>
> > +1
> >
> > In the Motivation section, you can quote the comment from pull
"WindowStoreIterator
fetch(K key,
long timeFrom, long timeTo)" to the WindowStore interface and make all the
ReadOnlyWindowStore methods take Durations. And likewise with the
SessionStore interfaces.
What do you think?
Thanks,
-John
On Fri, Aug 24, 2018 at 10:51 AM John Roesler wrote:
> Hi Nikol
://cwiki.apache.org/confluence/pages/diffpagesbyversion.action?pageId=87295409=11=9
Please let me know if this changes your vote.
Thanks,
-John
On Mon, Aug 13, 2018 at 5:20 PM John Roesler wrote:
> Hey all,
>
> I just wanted to let you know that a few small issues surfac
Just a quick thought regarding headers:
> I think there is no absolute-safe ways to avoid conflicts, but we can
still
> consider using some name patterns to reduce the likelihood as much as
> possible.. e.g. consider sth. like the internal topics naming: e.g.
> "__internal_[name]"?
I think there
Hi Nikolay,
Thanks for this very nice KIP!
To answer your questions:
1. Correct, we should not delete existing methods that have been released,
but ...
2. Yes, we should deprecate the 'long' variants so that we can drop them
later on. Personally, I like to mention which version deprecated the
+1 (non-binding)
On Sat, Aug 25, 2018 at 1:16 PM Joan Goyeau wrote:
> Hi,
>
> We want to make sure that we always have a serde for all Materialized,
> Serialized, Joined, Consumed and Produced.
> For that we can make use of the implicit parameters in Scala.
>
> KIP:
>
>
dowSize();
> > > }
> > >
> > > SessionBytesStoreSupplier {
> > > long retentionPeriod() -> Duration retentionPeriodDuration();
> > > }
> > >
> > > WindowBytesStoreSupplier {
> > > long windowSize() -> Duratio
ent #5539 (comment)
> > > > > <https://github.com/apache/kafka/pull/5539#discussion_r212380648>
> > > "This
> > > > > class was already defaulted to public visibility, and we can't
> > retract
> > > it
> > > > > now, without a K
Hi Flavien,
As far as I'm concerned, it's perfectly appropriate to keep pinging
periodically if no one responds to your ticket. I'm sorry no one commented
on your ticket or PR!
I have to admit that I don't know much about the coordinator, but I do have
one question I'll leave a comment on the
Please, take a look.
>
> Are we ready to vot now?
>
> В Ср, 29/08/2018 в 14:51 -0500, John Roesler пишет:
> > Hey Nikolay, sorry for the silence. I'm taking another look at the KIP
> > before voting...
> >
> >
> >1. I think the Window constructor should a
ed KIP-366 title?
>
> Thanks
>
> On Wed, 29 Aug 2018 at 21:14 John Roesler wrote:
>
> > Hey Joan,
> >
> > It looks like you've updated the KIP to "Accepted", but I only count one
> > binding vote (Guozhang). Ted, Attila, Bill, and myself are all
> n
I'm a +1 (non-binding)
On Mon, Sep 3, 2018 at 8:33 AM Nikolay Izhikov wrote:
> Dear commiters.
>
> Please, vote on a KIP.
>
> В Пт, 31/08/2018 в 12:05 -0500, John Roesler пишет:
> > Hi Nikolay,
> >
> > You can start a PR any time, but we cannot per it (and prob
I'm a +1 (non-binding) because we doubt the class is in use.
If you decide to copy it to a private version and deprecate the original
instead, as Matthias suggested, I would still be a +1.
Thanks,
-John
On Sat, Sep 1, 2018 at 6:47 AM Joan Goyeau wrote:
> Hi,
>
> As pointed out in this comment
vote.
> > Should I send again a new email with the updated KIP-366 title?
> >
> > Thanks
> >
> > On Wed, 29 Aug 2018 at 21:14 John Roesler wrote:
> >
> >> Hey Joan,
> >>
> >> It looks like you've updated the KIP to "Accepted"
gt; On Mon, Aug 27, 2018 at 5:20 PM, Dongjin Lee
> wrote:
> > >
> > >> +1 (non-binding)
> > >>
> > >> On Tue, Aug 28, 2018 at 8:53 AM Bill Bejeck
> wrote:
> > >>
> > >>> +1
> > >>>
> > >>> -
Hi James,
The proposal we are discussing is
https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+Support+non-key+joining+in+KTable
I'm not sure if it's been updated to reflect current thinking.
-John
On Fri, Sep 7, 2018 at 8:49 AM James Kwan wrote:
> I am new to this group and I found
Hey Nikolay, sorry for the silence. I'm taking another look at the KIP
before voting...
1. I think the Window constructor should actually be protected. I don't
know if we need a constructor that takes Instant, but if we do add one, it
should definitely be protected.
2. `long
for SAM.
>
> Thanks
>
> On Mon, 27 Aug 2018 at 15:41 John Roesler wrote:
>
> > Hey Joan,
> >
> > I was thinking more about this... Do any of the conversions in
> > FunctionConversions convert to types that are used in the public Scala
> > interface?
>
> > (4) Stores: 3 methods are listed as deprecated but only 2 new methods
> are added.
>
> My proposal based on John Roesler mail [1]:
> "10. Stores: I think we can just deprecate without replacement the method
> that takes `segmentInterval`."
>
> Is it wron
.
> If you need any help implementing this feature, please tell me.
>
> Thanks.
>
> -Flávio Stutz
>
>
>
>
> On 2018/07/03 18:04:52, John Roesler wrote:
> > Hi Flávio,
> > Thanks! I think that we can actually do this, but the API could be
> better.
files#diff-2fdec52b9cc3d0e564f0c12a199bed77
I have one working integration test, but it's slow going getting the timing
right, so no promises of any kind ;)
Let me know what you think!
Thanks,
-John
On Thu, Jul 5, 2018 at 8:39 AM John Roesler wrote:
> Hey Flávio,
>
> Thanks! I haven't got
d also address the issue about mental effort for "single
> > final window result" use case.
> >
> > I also think that a shorter close-time than retention time is useful for
> > window aggregation. If we add close() to the window definition and
> > until() to
value should we set?
>
> 2. For API names, some personal suggestions here:
>
> 2.a) "allowedLateness" -> "until" (semantics changed, and also value is
> defined as delta on top of window length), where "until" ->
> "retentionPeriod", and the
gt; but only to check that "windowSizeMs <= retentionPeriodMs". We can do this
> check at the store builder lever instead of at the processor level.
>
>
> If we can remove its usage in both 1) and 2), then we should be able to
> safely remove this from the `Windows` sp
this except document it well? Maybe log a
warning if we see that close wasn't explicitly set while using "final
results"?
Thanks,
-John
On Tue, Jul 10, 2018 at 10:46 AM John Roesler wrote:
> Hi Guozhang,
>
> That sounds good to me. I'll include that in the KIP.
>
>
Thank you!
I'm +1 (non-binding)
On Thu, Jul 5, 2018 at 10:24 PM Richard Yu
wrote:
> Nice KIP!
> +1 (non-binding)
> -Richard
>
> On Friday, July 6, 2018, 9:10:43 AM GMT+8, Matthias J. Sax <
> matth...@confluent.io> wrote:
>
> Thanks for the KIP!
>
> +1 (binding)
>
>
> -Matthias
>
> On
Hi Florian,
Sorry I'm late to the party, but I missed the message originally.
Regarding the names, it's probably a good idea to stick to the same
character set we're currently using: letters, numbers, and hyphens. The
names are used in Kafka topics, files and folders, and RocksDB databases,
and
d warning, and also
> given that we'd probably better not reusing the function name `until` for
> close time.
>
>
> Guozhang
>
>
> On Tue, Jul 10, 2018 at 3:31 PM, John Roesler wrote:
>
> > I had some opportunity to reflect on the default for close time today...
put the page in the incorrect location.
>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-331+Add+default+implementation+to+close%28%29+and+configure%28%29+for+Serializer%2C+Deserializer+and+Serde
>
> Cheers,
> Chia-Ping
>
> On 2018/07/02 19:45:19, John Roesler wrote:
> >
ed twice.
>
> I tried to think about using GlobalKTables, but I didn't get an easy way
> to aggregate the keys from that table. Do you have any clue?
>
> Thanks.
>
> -Flávio Stutz
>
>
>
>
>
>
> /partial-counters-to-single-partitioned-topic
>
> On 2018/
Dear Kafka community,
I am proposing KIP-267 to augment the public Streams test utils API.
The goal is to simplify testing of Kafka Streams applications.
Please find details in the
On Wed, Mar 7, 2018 at 8:03 PM, John Roesler <j...@confluent.io> wrote:
> Thanks Ted,
>
> Sure thing; I updated the example code in the KIP with a little snippet.
>
> -John
>
> On Wed, Mar 7, 2018 at 7:18 PM, Ted Yu <yuzhih...@gmail.com> wrote:
>
>> Looks
Hello all,
I'd like to start voting on KIP-267, to introduce a MockProcessorContent
enabling Processor, Transformer, and ValueTransformer authors to unit-test
their components.
k it is a good cost to pay,
> plus once we start publishing test-util artifacts for other projects like
> client and connect, we may face the same issue and need to do this
> refactoring as well.
>
>
>
> Guozhang
>
>
>
>
> On Fri, Mar 9, 2018 at 9:54 AM, John R
t;> update themselves.
> >>>>>
> >>>>> What do you think?
> >>>>>
> >>>>> The thing that made me think of this is that the "2 rolling bounces"
> is
> >>>> similar to what Kafka brokers have to do c
Hey Matthias,
The KIP looks good to me. I had several questions queued up, but they were
all in the "rejected alternatives" section... oh, well.
One very minor thought re changing the state directory from "//<
application.id>//rocksdb/storeName/" to "//<
application.id>//rocksdb-v2/storeName/":
> >>>> impression how the metadata upgrade works including a system test:
> >>>> https://github.com/apache/kafka/pull/4636
> >>>>
> >>>> I can share other PRs as soon as they are ready. I agree that the KIP
> is
> >>>> comple
> V value, String childName)` -- should we also throw
> UnsupportedOperationException similar to `schedule(long)` if KIP-251 is
> accepted?
>
>
> -Matthias
>
> On 3/8/18 3:16 PM, John Roesler wrote:
> > Thanks for the review, Guozhang,
> >
> > In response:
>
hn
On Thu, Mar 8, 2018 at 3:39 PM, Matthias J. Sax <matth...@confluent.io>
wrote:
> Isn't MockProcessorContext in o.a.k.test part of the unit-test package
> but not the main package?
>
> This should resolve the dependency issue.
>
> -Matthias
>
> On 3/8/18 3:32 P
hird module that depends on both
streams and test-utils. Yuck!
Thanks,
-John
On Thu, Mar 8, 2018 at 3:16 PM, John Roesler <j...@confluent.io> wrote:
> Thanks for the review, Guozhang,
>
> In response:
> 1. I missed that! I'll look into it and update the KIP.
>
> 2.
e punctuators are
> indeed registered, and if people want full auto punctuation testing they
> have to go with TopologyTestDriver.
>
>
>
> Guozhang
>
>
> On Wed, Mar 7, 2018 at 8:04 PM, John Roesler <j...@confluent.io> wrote:
>
> > On Wed, Mar 7, 2018 at 8:0
documentation
> as
> > to
> >why it's there.
> >2. Create a new package under o.a.k.test, called internals and move
> the
> >existing MockProcessorContext there, but that would require a change
> to
> > the
> >visibility of the MockProces
> > >>> the re-shuffling task is purely stateless and hence does not
> require
> > > "key
> > > >>> partitioning preservation". Operational-wise it is similar to the
> > > >>> "creating
> > > >>> a new topic w
wangg...@gmail.com>
> > wrote:
> > >
> > >> +1 (binding).
> > >>
> > >> On Mon, Apr 2, 2018 at 7:22 AM, Ted Yu <yuzhih...@gmail.com> wrote:
> > >>
> > >>> +1
> > >>>
> > >>> On Mon,
>
> > On Tue, Apr 3, 2018 at 1:35 PM, Bill Bejeck <bbej...@gmail.com> wrote:
> >
> >> Hi John,
> >>
> >> Thanks for making the updates.
> >>
> >> I agree with the information you've included in the logs as described
> >> above, as lo
Hi Vito,
Thanks for the KIP!
I think it's much nicer to give callers different exceptions to tell them
whether the state store got migrated, whether it's still initializing, or
whether there's some unrecoverable error.
In the KIP, it's typically not necessary to discuss non-user-facing details
Thanks again for this effort. I'm +1 (non-binding).
-John
On Mon, Apr 16, 2018 at 9:39 AM, Ismael Juma wrote:
> Thanks for the contribution. I haven't reviewed all the new APIs in detail,
> but the general approach sounds good to me. +1 (binding).
>
> Ismael
>
> On Wed, Apr
n this thread to let each other know (makes
> it easier to coordinate progress).
>
> Thanks,
> Richard
>
> On Tue, Apr 17, 2018 at 2:07 PM, John Roesler <j...@confluent.io> wrote:
>
> > Ok, I'll close the discussion on KIP-288 and mark it discarded.
> >
> > We
Hey Richard,
As you noticed, the newly introduced KIP-288 overlaps with this one. Sorry
for stepping on your toes... How would you like to proceed? I'm happy to
"close" KIP-288 in deference to this KIP.
With respect to poll(), reading this discussion gave me a new idea for
providing a
Ah, I see that this KIP is a sub-kip of KIP-266. Sorry about that; I've
sent a message to the discussion thread of KIP-266 to work this out.
Thanks,
-John
On Tue, Apr 17, 2018 at 1:29 PM, John Roesler <j...@confluent.io> wrote:
> Thanks, Richard, I'll check it out.
>
> On Tue, Ap
"operation timeout" as a separate parameter from
the "long-poll time". Or maybe adding an "asyncPoll(timeout, time unit)"
that only uses the timeout to bound metadata updates and otherwise behaves
like the current "poll(0)".
Thanks,
-John
On Tue, Apr 17
1 - 100 of 1190 matches
Mail list logo