+1
On Thu, Jun 25, 2020 at 6:39 PM Bill Bejeck wrote:
> Thanks for this KIP Xavier.
>
> +1(binding)
>
> -Bill
>
> On Thu, Jun 25, 2020 at 9:04 PM Gwen Shapira wrote:
>
> > +1 (binding)
> >
> > Thank you Xavier!
> >
> > On Thu, Jun 25, 2020, 3:44 PM Xavier Léauté wrote:
> >
> > > Hi Everyone,
>
ether it is safe to just read the latest segment without
> sanity checking all previous inactive segment of a given partition if
> transaction is used. Otherwise we probably want to always skip the sanity
> check of inactive segments without introducing a new config. Maybe the
>
Optimizing startup seems really valuable but I'm a little confused by this.
There are two different things:
1. Recovery
2. Sanity check
The terminology we're using is a bit mixed here.
Recovery means checksumming the log segments and rebuilding the index on a
hard crash. This only happens on unf
Hey Jason,
I agree. Even apart from this proposal the partitioning strategy is really
an essential part of the metadata for a topic and had we been less lazy we
probably would have included it with the topic metadata.
I think in terms of grandfathering this in you could have existing topics
just
+1
I think we can improve this in the future, but this simple change will
avoid a lot of pain. Thanks for reviving it Ewen.
-Jay
On Mon, Mar 5, 2018 at 11:35 AM, Ewen Cheslack-Postava
wrote:
> I'd like to kick off voting for KIP-186:
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> 1
he topic anyway even with
> this optimization. I think this algorithm is kind of orthogonal to this
> KIP. We can extend the KIP to support this algorithm in the future as well.
>
> Thanks,
> Dong
>
> On Thu, Feb 22, 2018 at 5:19 PM, Jay Kreps wrote:
>
> > Hey Dong
Hey Dong,
Two questions:
1. How will this work with Streams and Connect?
2. How does this compare to a solution where we physically split partitions
using a linear hashing approach (the partition number is equivalent to the
hash bucket in a hash table)? https://en.wikipedia.org/wiki/Linear_hashing
Two thoughts on implementation (shouldn't effect the KIP):
1. It might be nice to add a parameter to ConfigDef which says whether a
configuration is dynamically updatable or not so that we can give error
messages if it isn't and also have it reflected in the auto-generated docs.
2. For
a little more might be okay.
-Jay
On Wed, Nov 22, 2017 at 1:46 PM, Colin McCabe wrote:
> On Wed, Nov 22, 2017, at 13:43, Colin McCabe wrote:
> > On Wed, Nov 22, 2017, at 13:08, Jay Kreps wrote:
> > > Okay yeah, what I said didn't really work or make sense. Ismael's
&g
n it
from scratch in such a way that it is obvious you'd have these two things
as the fundamental primitives for fetching data.
-Jay
On Wed, Nov 22, 2017 at 11:02 AM, Colin McCabe wrote:
> Hi Jay,
>
> On Tue, Nov 21, 2017, at 19:03, Jay Kreps wrote:
> > I think the gen
I think the general thrust of this makes a ton of sense.
I don't love that we're introducing a second type of fetch request. I think
the motivation is for compatibility, right? But isn't that what versioning
is for? Basically to me although the modification we're making makes sense,
the resulting
[
https://issues.apache.org/jira/browse/KAFKA-414?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Jay Kreps resolved KAFKA-414.
-
Resolution: Won't Fix
> Evaluate mmap-based writes for Log implem
Hey Clebert,
Is there a motivation for adding a second way? We generally try to avoid
having two ways to do something unless it's really needed...I suspect you
have a reason for wanting this, though.
-Jay
On Mon, Oct 2, 2017 at 6:15 AM Clebert Suconic
wrote:
> At ActiveMQ and ActiveMQ Artemis,
Yes the idea of the correlation id is to make it easier for the client to
match a particular response to the request it answers. Kafka’s protocol
allows sending multiple requests without waiting for the response. In
theory you can just rely on ordering, but that can be a bit fragile if the
client h
Hey Guozhang,
I think the argument is that with acks=1 the message could be lost and
hence you aren't guaranteeing exactly once delivery.
-Jay
On Mon, Aug 14, 2017 at 1:36 PM, Guozhang Wang wrote:
> Just want to clarify that regarding 1), I'm fine with changing it to `all`
> but just wanted to
Becket,
I think this proposal actually does a great deal to address the
configuration complexity. It is true that there are a number of knobs, but
the result of this change is that 99% of people don't need to think about
them (and the mechanism we have to communicate that is to reduce the
importan
+1
I think there is some pain with changing any default, but this is well
worth it.
The reality is that the profile of Kafka usage has changed significantly
since replication was added to Kafka and these defaults were chosen. At
that time usage was primarily high volume event and log data and the
1.0! Let's do it!
-Jay
On Tue, Jul 18, 2017 at 3:36 PM, Guozhang Wang wrote:
> Hi all,
>
> With 0.11.0.0 out of the way, I would like to volunteer to be the
> release manager
> for our next time-based feature release. See https://cwiki.apache.org/
> confluence/display/KAFKA/Time+Based+Release+P
I suspect this is on Linux right?
The way Linux works is it uses a percent of memory to buffer new writes, at
a certain point it thinks it has too much buffered data and it gives high
priority to writing that out. The good news about this is that the writes
are very linear, well layed out, and hig
+1
On Sat, May 27, 2017 at 11:04 AM, Gwen Shapira wrote:
> Thanks Vahid,
>
> Do you mind if we leave the command-line out of scope for this?
>
> I can see why adding confirmations, options to bypass confirmations, etc
> would be an improvement. However, I've seen no complaints about the current
I think the question is when do you actually *want* processing time
semantics? There are definitely times when its safe to assume the two are
close enough that a little lossiness doesn't matter much but it is pretty
hard to make assumptions about when the processing time is and has been
hard for us
I think the question is when do you actually *want* processing time
semantics? There are definitely times when its safe to assume the two are
close enough that a little lossiness doesn't matter much but it is pretty
hard to make assumptions about when the processing time is and has been
hard for us
+1
-Jay
On Thu, Jun 8, 2017 at 11:16 AM, Guozhang Wang wrote:
> I think we can continue on this voting thread.
>
> Currently we have one binding vote and 2 non-binging votes. I would like to
> call out for other people especially committers to also take a look at this
> proposal and vote.
>
>
>
[
https://issues.apache.org/jira/browse/KAFKA-1955?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16043494#comment-16043494
]
Jay Kreps edited comment on KAFKA-1955 at 6/8/17 9:43 PM:
-
[
https://issues.apache.org/jira/browse/KAFKA-1955?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16043494#comment-16043494
]
Jay Kreps edited comment on KAFKA-1955 at 6/8/17 9:41 PM:
-
[
https://issues.apache.org/jira/browse/KAFKA-1955?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16043494#comment-16043494
]
Jay Kreps commented on KAFKA-1955:
--
I think the patch I submitted was kind of a
I think Ram's point is that in place failure is pretty complicated, and
this is meant to be a cost saving feature, we should construct an argument
for it grounded in data.
Assume an annual failure rate of 1% (reasonable, but data is available
online), and assume it takes 3 days to get the drive re
Hey Eno,
I think this makes sense. I do think people who spend time running
production stream processing systems will, over time, end up strongly
preferring the current behavior of failing and fixing the root problem
rather than skipping, but we don't need to force this on people as long as
the de
+1
On Tue, May 9, 2017 at 3:41 PM BigData dev wrote:
> Hi, Everyone,
>
> Since this is a relatively simple change, I would like to start the voting
> process for KIP-156: Add option "dry run" to Streams application reset tool
>
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=69
+1
On Fri, May 5, 2017 at 7:29 PM Sriram Subramanian wrote:
> +1
>
> On Fri, May 5, 2017 at 6:04 PM, Gwen Shapira wrote:
>
> > +1
> >
> > On Fri, May 5, 2017 at 3:32 PM, Ismael Juma wrote:
> >
> > > Hi all,
> > >
> > > Given the simple and non controversial nature of the KIP, I would like
> to
I haven't digested the proposal but the use case is pretty common. An
example would be the "customer 360" or "unified customer profile" use case
we often use. In that use case you have a dozen systems each of which has
some information about your customer (account details, settings, billing
info, c
IP 141 - ProducerRecordBuilder Interface
> >
> > I understand that we cannot just break stuff (btw: also not for
> > Streams!). But deprecating does not break anything, so I don't think
> > it's a big deal to change the API as long as we keep the old API
d constructors,
> too.
>
> Just my two cents.
>
>
> -Matthias
>
> On 4/23/17 3:37 PM, Jay Kreps wrote:
> > Hey guys,
> >
> > I definitely think that the constructors could have been better designed,
> > but I think given that they're in heavy us
Congrats Rajini!
On Mon, Apr 24, 2017 at 2:06 PM, Gwen Shapira wrote:
> The PMC for Apache Kafka has invited Rajini Sivaram as a committer and we
> are pleased to announce that she has accepted!
>
> Rajini contributed 83 patches, 8 KIPs (all security and quota
> improvements) and a significant n
Hey guys,
I definitely think that the constructors could have been better designed,
but I think given that they're in heavy use I don't think this proposal
will improve things. Deprecating constructors just leaves everyone with
lots of warnings and crossed out things. We can't actually delete the
+1 Very well thought out.
-Jay
On Fri, Apr 21, 2017 at 10:39 AM Eno Thereska
wrote:
> Hi there,
>
> Unless there are more issues on the discuss thread, I'd like to start the
> vote on KIP-114.
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-114%3A+KTable+state+stores+and+improved+semant
Hey guys,
One thing I've always found super important for this kind of design work is
to do a really good job of cataloging the landscape of use cases and how
prevalent each one is. By that I mean not just listing lots of uses, but
also grouping them into categories that functionally need the same
+1
-Jay
On Mon, Mar 20, 2017 at 11:27 AM, Matthias J. Sax
wrote:
> Hi,
>
> I would like to start the vote for KIP-129. Of course, feel free to
> provide some more feedback on the DISCUSS thread.
>
> Thanks a lot!
>
>
> -Matthias
>
>
ink
> > > >>>>> about good names. It's just to discuss the pattern.
> > > >>>>>
> > > >>>>>
> > > >>>>>
> > > >>>>> -Matthias
> > > >
pe for this KIP. KIP-120 has the focus on removing leaking
> internal APIs and do some cleanup how our API reflects some concepts.
>
> However, I added your idea to API discussion Wiki page and we take if
> from there:
> https://cwiki.apache.org/confluence/display/KAFKA/
> Kafka+S
Two things:
1. This is a minor thing but the proposed new name for KStreamBuilder
is StreamsTopologyBuilder. I actually think we should not put topology in
the name as topology is not a concept you need to understand at the
kstreams layer right now. I'd think of three categories of con
I second the concern on with the one producer per task approach. At a
high-level it seems to make sense but I think Damian is exactly right that
that cuts against the general design of the producer. Many people have high
input partition counts and will have high task counts as a result. I think
pro
; > Dong
> >> > >
> >> > >
> >> > > On Thu, Feb 23, 2017 at 9:25 AM, Jun Rao wrote:
> >> > >
> >> > >> Hi, Ismael,
> >> > >>
> >> > >> For #3, typically, an admin won't config
the consumer now can fetch oversized
> messages. One concern would be that on the broker side oversized messages
> will bring more memory pressure. With KIP-92, we may mitigate that, but the
> memory allocation for large messages may not be very GC friendly. I need to
> think about this a li
; durations, but would result in more meaningful ratios. We
> > > could
> > > > > > > define
> > > > > > > > > two
> > > > > > > > > > quota limits (UserA has 5% of request threads and 10% of
> > > > network
Hey Becket,
I get the problem we want to solve with this, but I don't think this is
something that makes sense as a user controlled knob that everyone sending
data to kafka has to think about. It is basically a bug, right?
First, as a technical question is it true that using the uncompressed size
se something like "your request
> rate quota on metadata request has reached".
>
> Thanks,
>
> Jiangjie (Becket) Qin
>
>
> On Mon, Feb 20, 2017 at 2:23 PM, Jay Kreps wrote:
>
> > I think this proposal makes a lot of sense (especially now that it is
>
I think this proposal makes a lot of sense (especially now that it is
oriented around request rate) and fills the biggest remaining gap in the
multi-tenancy story.
I think for intra-cluster communication (StopReplica, etc) we could avoid
throttling entirely. You can secure or otherwise lock-down t
+1
Super happy with how this turned out. It's been a long journey since we
started thinking about this 3+ years ago. Can't wait to see it in
code---this is a big one! :-)
-Jay
On Wed, Feb 1, 2017 at 8:13 PM, Guozhang Wang wrote:
> Hi all,
>
> We would like to start the voting process for KIP-9
+1
Nice improvement.
-Jay
On Tue, Feb 14, 2017 at 1:22 PM, Steven Schlansker <
sschlans...@opentable.com> wrote:
> Hi, it looks like I have 2 of the 3 minimum votes, can a third voter
> please consider this KIP?
> Thanks.
>
> (PS - new revision on GitHub PR with hopefully the last round of
> im
Couple of things I think we still need to work out:
1. I think we agree about the key, but I think we haven't talked about
the value yet. I think if our goal is an open ecosystem of these header
spread across many plugins from many systems we should consider making this
a string as wel
don't agree on what is complex and what is simple. :) For example, I
> > think
> > batch APIs are a significant source of complexity as you have to do a
> > bunch
> > of ceremony to group things before sending the request and error handling
> > becomes more com
now Rust, take
pity on me! :-)
Nit picking aside, super excited to see us progress on this.
-Jay
On Tue, Feb 7, 2017 at 3:46 PM Ismael Juma wrote:
> Hi Jay,
>
> Thanks for the feedback. Comments inline.
>
> On Tue, Feb 7, 2017 at 8:18 PM, Jay Kreps wrote:
> >
> >
Hey Colin,
This is great, thought I'd throw out a couple of opinions to the mix, feel
free to ignore:
- I think it would be good to not use "get" as the prefix for things
making remote calls. We've tried to avoid the java getter convention
entirely (see code style guide), but for remote
Congrats Grant!
-Jay
On Wed, Jan 11, 2017 at 11:51 AM, Gwen Shapira wrote:
> The PMC for Apache Kafka has invited Grant Henke to join as a
> committer and we are pleased to announce that he has accepted!
>
> Grant contributed 88 patches, 90 code reviews, countless great
> comments on discussion
+1
On Wed, Jan 11, 2017 at 10:56 AM, Ben Stopford wrote:
> Looks like there was a good consensus on the discuss thread for KIP-106 so
> lets move to a vote.
>
> Please chime in if you would like to change the default for
> unclean.leader.election.enabled from true to false.
>
> https://cwiki.apa
This is great! A couple of quick comments:
1. It'd be good to make the motivation a bit more clear. I think the
motivation is "We want to have lots of partition/task/etc metrics but we're
concerned about the performance impact so we want to disable them by
default." Currently the motiv
Hey Jun,
We had a proposal like this previously. The suppression scheme was slightly
different. Rather than than attempting to recopy or swap, there was instead
an aborted offset index maintained along with each segment containing a
sequential list of aborted offsets. The filtering would happen at
+1
On Tue, Dec 20, 2016 at 1:01 PM, Hojjat Jafarpour
wrote:
> Hi all,
>
> Seems that there is no opposition to this KIP. This email is to start the
> voting for this KIP.
> Once again the KIP is for removing zkClient dependency from Streams. Please
> check out the KIP page:
>
> https://cwiki.apa
the amount of time
> from the moment the producer started the TX to the time when it was
> committed. in my mind this time span is very short. am I failing to
> understand the proposed "typical" use case? is the plan to use long-running
> transactions and only commit at, say, 5 mi
; > "transaction ordering", that is, consumer will not see messages from
> the
> > > same partition in the order where they were produced, but only in the
> > order
> > > of when the corresponding transaction was committed. For some
> scenarios,
> > we
> > &
I think quite a few
> guys at LinkedIn are quite pro it too ;) I'm happy to drop this one.
>
> Cheers
> Mike
>
> Sent using OWA for iPhone
>
> From: Jay Kreps
> Sent: Monday, December 19, 2016 8:51:23 PM
> To: dev@kafka.apa
> constructive on discussion of the implementation detail.
>
> kip-87 spawned from as on the kip call we all agreed this was needed. And
> would at least allow a custom wrapper be supported in a compacted topic,
> allowing meta data. Which again now I feel we are spinning wheels, a
because there is no uncommitted messages
> > exposed to the consumers. This may potentially give us a chance to
> > significantly improve consumer side efficiency in normal cases. It again
> > depends on the use case, i.e. whether user can process a transaction
> > progressivel
I think this is great. Sounds like one implication is that existing code
that called close() and hit the timeout would now hang indefinitely. We saw
this kind of thing a lot in automated testing scenarios where people don't
correctly sequence their shutdown of client and server. I think this is
oka
Yeah good point. I relent!
-jay
On Fri, Dec 16, 2016 at 1:46 PM Jason Gustafson wrote:
> Jay/Ismael,
>
>
>
> I agree that lazy initialization of metadata seems unavoidable. Ideally, we
>
> could follow the same pattern for transactions, but remember that in the
>
> consumer+producer use case, t
e on discussion of the implementation detail.
>
> kip-87 spawned from as on the kip call we all agreed this was needed. And
> would at least allow a custom wrapper be supported in a compacted topic,
> allowing meta data. Which again now I feel we are spinning wheels, and
> simply fin
P-71
> > to maintain a hybrid compacted/retention topic.
> >
> > -Ewen
> >
> > On Sun, Dec 11, 2016 at 1:18 PM, Michael Pearce <
> michael.pea...@ig.com>
> > wrote:
> >
> > > Hi Jay,
> > >
>
Ha, least controversial KIP ever. :-)
-Jay
On Tue, Dec 13, 2016 at 10:39 AM, Hojjat Jafarpour
wrote:
> Hi all,
>
> The following is a KIP for removing zkClient dependency from Streams.
> Please check out the KIP page:
>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> 90+-+Remove+zkCl
mer nice symmetry
> with
> > > the producer and its generic naming gives us some flexibility
> internally
> > > with how we use it. We could still do that I guess, if we're willing to
> > > deprecate `fetch.max.bytes` (one release after adding it!).
> > >
of having a single `init`, but I am not sure about the
> specifics of the metadata initialisation (as Jason alluded to). More
> inline.
>
> On Tue, Dec 13, 2016 at 6:01 AM, Jay Kreps wrote:
>
> >1. Add a generic init() call which initializes both transactions and
>
t; in favor of `fetch.max.bytes`.
>
> -Jason
>
> On Sat, Dec 10, 2016 at 2:27 PM, Jay Kreps wrote:
>
> > Jason, it's not just decompression but also the conversion from packed
> > bytes to java objects, right? That can be even larger than the
> > decompre
Hey Jason/Neha,
Yeah, clearly having a mandatory, generic init() method that initializes
both transactions and topic metadata would be the ideal solution. This
would solve the occasional complaint about blocking behavior during
initialization of metadata (or at least shift it to a new complaint ab
e losing the schema truly is losing information
> (whereas null is already the only valid value for a pure null schema).
>
> -Ewen
>
>
> On Sat, Dec 10, 2016 at 9:24 PM, Michael Pearce
> wrote:
>
> > Hi Jay,
> >
> > Good point this detail is missing in the KIP write up. Ive added this
> now.
Michael,
The compatibility section goes through the migration path, but isn't the
bigger compatibility issue with existing apps? There are many (probably
thousands) of apps in production that use this feature and send null to
mean delete. It seems like this would break compatibility with them, and
+1 (binding)
-Jay
On Wed, Dec 7, 2016 at 9:17 AM, Colin McCabe wrote:
> Hi all,
>
> I heard that the VOTE and DISCUSS threads for the KIP-97 discussion
> appeared to be in the same email thread for some people using gmail. So
> I'm reposting in hopes of getting a separate email thread this tim
Jason, it's not just decompression but also the conversion from packed
bytes to java objects, right? That can be even larger than the
decompression blow up. I think this may be okay, the problem may just be
that the naming is a bit misleading. In the producer you are literally
allocating a buffer o
rlier, is there a better name to describe this
> feature, if we're not implementing transactions to the traditional
> transaction expected, to avoid this confusion?
>
>
> Sent using OWA for iPhone
>
> From: Jay Kreps
> Sent: Friday,
k as they're used to.
>
> As my other response earlier, is there a better name to describe this
> feature, if we're not implementing transactions to the traditional
> transaction expected, to avoid this confusion?
>
>
> Sent using OWA for iPhone
> ___
s
> for performance and simplicity. but we do alas unfortunately have one or
> two places we cannot ignore it.
>
> Cheers
> Mike
>
> Sent using OWA for iPhone
>
> From: Jay Kreps
> Sent: Thursday, December 8, 2016 11:25:53
Hey Edoardo,
For (3) can you outline what you think the benefit and use cases for a more
general cross-system XA feature would be an what changes to the proposal
would be required to enable it? When I have asked people who wanted
cross-system XA in the past what they wanted it for, I haven't reall
de, I can say that the benefit of having a
> > conceptually simpler message format is huge. It allows you to converge
> the
> > paths for validation of message sets on the broker, for example.
> Currently,
> > we pretty much need two separate paths everywhere we process messages. It
I'd like to second the discouragement of adding a new topic per job. We
went down this path in Samza and I think the result was quite a mess. You
had to read the full topic every time a job started and so it added a lot
of overhead and polluted the topic space.
What if we did the following:
1.
Looks great!
A few questions:
1. What is the relationship between transaction.app.id and the existing
config application.id in streams?
2. The initTransactions() call is a little annoying. Can we get rid of
that and call it automatically if you set a transaction.app.id when we
do t
[
https://issues.apache.org/jira/browse/KAFKA-1941?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Jay Kreps resolved KAFKA-1941.
--
Resolution: Duplicate
Yes, I think this is resolved by KAFKA-1282.
> Timeout connections in
Hey Radai,
+1 on deprecating and eventually removing the old config. The intention was
absolutely bounding memory usage. I think having two ways of doing this,
one that gives a crisp bound on memory and one that is hard to reason about
is pretty confusing. I think people will really appreciate hav
t; >
> > > On Sun, Oct 30, 2016 at 12:32 PM, Jun Rao wrote:
> > >
> > > > Hi, Mickael,
> > > >
> > > > I agree with others that it's better to be able to control the bytes
> > the
> > > > consumer can read from sockets, inst
Xavier,
Yeah I think that post KIP-58 it is possible to depend on the delivery of
messages in compacted topics, if you override the default compaction time.
Prior to that it was true that you could control the delete retention, but
any message including the tombstone could be compacted away prior
This is a good observation on limiting total memory usage. If I understand
the proposal I think it is that the consumer client would stop sending
fetch requests once a certain number of in-flight fetch requests is met. I
think a better approach would be to always issue one fetch request to each
bro
I kind of agree with James that it is a bit questionable how valuable any
data in a delete marker can be since it will be deleted somewhat
nondeterministically.
Let's definitely ensure the change is worth the resulting pain and
additional complexity in the data model.
I think the two things we ma
-1
I think the REST server for Kafka that already exists is quite good and
getting contributions. Moving this into the core project doesn't solve a
problem that I see.
-Jay
On Tue, Oct 25, 2016 at 2:16 PM, Harsha Chintalapani
wrote:
> Hi All,
>We are proposing to have a REST Server
>> > > Mike
> >> > >
> >> > >
> >> > >
> >> > > From: Harsha Ch
> >> > > Sent: Thursday, October 20, 2016 10:26 PM
> >> > > To: dev@kafka.apache.org
by community
> > aspect, though it's alike to ideal world. KIP makes innovation on
> adopting
> > new feature transparently, which makes many developers inspiring and
> > adopting it to their projects. Hopes that Kafka community continuously
> > drives the transpar
irrorMaker since they're definitely
> not used by all? I realize I say this even though at LinkedIn we have a
> REST setup of our own, just speaking from a community perspective.
>
> Nacho
>
>
> (I'm relatively new and I haven't read all of the mail archive, so I
Does adding a REST proxy to Apache Kafka make us more agile and
> > >> maintain
> > >> > the high-quality experience that Kafka users have today?
> > >> >
> > >> > For 1, I don't think there is value in giving in to the NIH syndrome
> >
Hey guys,
This discussion has come up a number of times and we've always passed.
One of things that has helped keep Kafka simple is not adding in new
abstractions and concepts except when the proposal is really elegant and
makes things simpler.
Consider three use cases for headers:
1. Kafka-
users to deal with less number
> of distributions/builds.
>
> Thanks,
> Manikumar
>
>
> On Sat, Oct 1, 2016 at 4:24 AM, Jay Kreps wrote:
>
> > Hey guys,
> >
> > There's already a REST interface maintained as a separate project--it's
> >
Hey guys,
There's already a REST interface maintained as a separate project--it's
open source and apache licensed and actively maintained (
https://github.com/confluentinc/kafka-rest). What is wrong with that? You
mentioned that there was some compatibility concern, but compatibility has
to do wit
Hey guys,
Gwen and I took a stab at better integrating Connect and Streams in the
Kafka site...they were largely absent in the api section, intro, home page,
etc. Take a look and see what you think. Major changes are the following:
- Changed tag line from "a distributed messaging system" to "a
This looks great, big improvements for the list offset protocol which is
currently quite odd.
One minor thing. I think the old v0 list offsets request also gave you the
highwater mark, it kind of shoves it in as the last thing in the array of
offsets. This is used internally to implement seekToEnd
1 - 100 of 2481 matches
Mail list logo