Re: [VOTE] KIP-797: Accept duplicate listener on port for IPv4/IPv6

2021-11-22 Thread Josep Prat
Hi Matthew,

Thank you for the PR.

+1 (non binding) from my side.


Best,

———
Josep Prat

Aiven Deutschland GmbH

Immanuelkirchstraße 26, 10405 Berlin

Amtsgericht Charlottenburg, HRB 209739 B

Geschäftsführer: Oskari Saarenmaa & Hannu Valtonen

m: +491715557497

w: aiven.io

e: josep.p...@aiven.io

On Tue, Nov 23, 2021, 07:11 Ivan Yurchenko  wrote:

> Hi,
>
> Thank you for the KIP.
>
> +1 (non-binding)
>
> Ivan
>
>
> On Tue, 23 Nov 2021 at 04:18, Luke Chen  wrote:
>
> > Hi Matthew,
> > Thanks for the KIP.
> > It makes sense to allow IPv4 and IPv6 listening on the same port for the
> > listener config.
> >
> > +1 (non-binding)
> >
> > Thank you.
> > Luke
> >
> > On Mon, Nov 22, 2021 at 6:28 PM Matthew de Detrich
> >  wrote:
> >
> > > Hello everyone,
> > >
> > > I would like to start a vote for KIP-797: Accept duplicate listener on
> > port
> > > for IPv4/IPv6
> > >
> > >
> >
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=195726330
> > >
> > > The purpose of the KIP is to loosen current validation for non
> advertised
> > > listeners so that you can have an IPv4 address and an IPv6 address on
> the
> > > same port. All other behaviour remains the same as before (since these
> > are
> > > disparate IP stacks there are no technical reasons not to allow this).
> > >
> > > PR is located at https://github.com/apache/kafka/pull/11478
> > >
> > > Comments and feedback are welcome!
> > >
> > > Regards
> > >
> > > --
> > >
> > > Matthew de Detrich
> > >
> > > *Aiven Deutschland GmbH*
> > >
> > > Immanuelkirchstraße 26, 10405 Berlin
> > >
> > > Amtsgericht Charlottenburg, HRB 209739 B
> > >
> > > Geschäftsführer: Oskari Saarenmaa & Hannu Valtonen
> > >
> > > *m:* +491603708037
> > >
> > > *w:* aiven.io *e:* matthew.dedetr...@aiven.io
> > >
> >
>


Re: [VOTE] KIP-797: Accept duplicate listener on port for IPv4/IPv6

2021-11-22 Thread Ivan Yurchenko
Hi,

Thank you for the KIP.

+1 (non-binding)

Ivan


On Tue, 23 Nov 2021 at 04:18, Luke Chen  wrote:

> Hi Matthew,
> Thanks for the KIP.
> It makes sense to allow IPv4 and IPv6 listening on the same port for the
> listener config.
>
> +1 (non-binding)
>
> Thank you.
> Luke
>
> On Mon, Nov 22, 2021 at 6:28 PM Matthew de Detrich
>  wrote:
>
> > Hello everyone,
> >
> > I would like to start a vote for KIP-797: Accept duplicate listener on
> port
> > for IPv4/IPv6
> >
> >
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=195726330
> >
> > The purpose of the KIP is to loosen current validation for non advertised
> > listeners so that you can have an IPv4 address and an IPv6 address on the
> > same port. All other behaviour remains the same as before (since these
> are
> > disparate IP stacks there are no technical reasons not to allow this).
> >
> > PR is located at https://github.com/apache/kafka/pull/11478
> >
> > Comments and feedback are welcome!
> >
> > Regards
> >
> > --
> >
> > Matthew de Detrich
> >
> > *Aiven Deutschland GmbH*
> >
> > Immanuelkirchstraße 26, 10405 Berlin
> >
> > Amtsgericht Charlottenburg, HRB 209739 B
> >
> > Geschäftsführer: Oskari Saarenmaa & Hannu Valtonen
> >
> > *m:* +491603708037
> >
> > *w:* aiven.io *e:* matthew.dedetr...@aiven.io
> >
>


Re: [VOTE] KIP-798 Add possibility to write kafka headers in Kafka Console Producer

2021-11-22 Thread Luke Chen
Hi Florin,
Thanks for the update!

+1 (non-binding)

Thank you.
Luke

On Tue, Nov 23, 2021 at 2:00 AM Florin Akermann 
wrote:

> Hi Bill and David,
>
> Thank you both for the vote.
> @David: KIP is updated.
>
> Florin
>
> On Mon, 22 Nov 2021 at 18:28, David Jacot 
> wrote:
>
> > Hi Florin,
> >
> > Thanks for the KIP. I am +1 (binding).
> >
> > There is a small typo in the Proposed Changes section:
> > `parse.header` should be `parse.headers`.
> >
> > Best,
> > David
> >
> > On Mon, Nov 22, 2021 at 6:20 PM Bill Bejeck  wrote:
> > >
> > > Hi Florin,
> > >
> > > Thanks for the KIP, this seems like a very useful addition.
> > >
> > > +1(binding).
> > >
> > > -Bill
> > >
> > > On Mon, Nov 22, 2021 at 12:00 PM Florin Akermann <
> > florin.akerm...@gmail.com>
> > > wrote:
> > >
> > > > Hi Luke and Tom
> > > >
> > > > @Tom: Thanks for the vote.
> > > >
> > > > @Luke: Thanks for the feedback.
> > > >
> > > > I have updated the KIP accordingly with regards to your comments on
> the
> > > > remaining case (false,false) and the motivation.
> > > >
> > > > Regarding the "not only UTF-8": As far as I understand John it is
> fine
> > to
> > > > limit the scope for this change to UTF-8 only as it is a handy
> > addition on
> > > > its own. Other formats can be relatively easily supported by adding
> > more
> > > > properties in later KIPs. In my reply to John (email from 21 Nov
> 2021,
> > > > 11:29 UTC) I also added an explanation why I limited the scope to
> UTF-8
> > > > only.
> > > >
> > > > Thanks,
> > > > Florin
> > > >
> > > >
> > > >
> > > > On Mon, 22 Nov 2021 at 10:32, Tom Bentley 
> wrote:
> > > >
> > > > > Hi Florin,
> > > > >
> > > > > Thanks for the KIP!
> > > > >
> > > > > +1 (binding),
> > > > >
> > > > > Kind regards,
> > > > >
> > > > > Tom
> > > > >
> > > > > On Mon, Nov 22, 2021 at 6:51 AM Luke Chen 
> wrote:
> > > > >
> > > > > > Hi Florin,
> > > > > > Thanks for the KIP.
> > > > > >
> > > > > > This KIP makes sense to me. Just a comment that the motivation
> > section
> > > > is
> > > > > > not clearly explain why this KIP is important.
> > > > > > I think John already mentioned a good motivation, which is to
> > support
> > > > > "not
> > > > > > only UTF-8".
> > > > > > You should put that into the KIP, and of course if you have other
> > > > > thoughts,
> > > > > > please also add them into KIP.
> > > > > >
> > > > > > Also, in the "public interface" section, there are 3 "Default
> > parsing
> > > > > > pattern", I think you should add 1 remaining case (false, false)
> to
> > > > make
> > > > > it
> > > > > > complete.
> > > > > >
> > > > > > Otherwise, look good to me.
> > > > > >
> > > > > > Thank you.
> > > > > > Luke
> > > > > >
> > > > > >
> > > > > > On Sun, Nov 21, 2021 at 7:37 PM Florin Akermann <
> > > > > florin.akerm...@gmail.com
> > > > > > >
> > > > > > wrote:
> > > > > >
> > > > > > > Hi John,
> > > > > > >
> > > > > > > Thanks for the vote and feedback.
> > > > > > >
> > > > > > > The thought occurred to me too.
> > > > > > >
> > > > > > > Do I understand it correctly: the current version of the
> > > > > > > kafka-console-producer cannot be used for anything other than
> > UTF-8
> > > > > keys
> > > > > > > and values?
> > > > > > > (There is no other implementation of MessageReader other than
> the
> > > > > > > ConsoleProducer$LineMessageReader)
> > > > > > > In other words, currently users seem to only apply it with
> utf-8
> > > > > strings
> > > > > > > for keys and values?
> > > > > > > This is why I figured I would not deviate from this assumption
> > solely
> > > > > for
> > > > > > > the headers.
> > > > > > >
> > > > > > > I will happily raise another KIP / Jira if there is a need to
> > specify
> > > > > > other
> > > > > > > formats / serializers for headers, keys and/or values.
> > > > > > >
> > > > > > > Thanks,
> > > > > > > Florin
> > > > > > >
> > > > > > >
> > > > > > > On Sat, 20 Nov 2021 at 19:34, John Roesler <
> vvcep...@apache.org>
> > > > > wrote:
> > > > > > >
> > > > > > > > Hi Florin,
> > > > > > > >
> > > > > > > > Thanks for the KIP!
> > > > > > > >
> > > > > > > > I think the assumption that header values are UTF-8 strings
> > might
> > > > not
> > > > > > > hold
> > > > > > > > up in the long run, but it seems like we can easily add a
> > property
> > > > > > later
> > > > > > > to
> > > > > > > > specify the format. It seems like this scope is probably a
> > handy
> > > > > > addition
> > > > > > > > on its own.
> > > > > > > >
> > > > > > > > I’m +1 (binding)
> > > > > > > >
> > > > > > > > Thanks,
> > > > > > > > John
> > > > > > > >
> > > > > > > >
> > > > > > > > On Fri, Nov 19, 2021, at 15:06, flo wrote:
> > > > > > > > > <
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-798%3A+Add+possibility+to+write+kafka+headers+in+Kafka+Console+Producer
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> >
>


Re: [VOTE} KIP-796: Interactive Query v2

2021-11-22 Thread John Roesler
Thanks for voting and for the discussion, all!

The vote on KIP-796 passes with:
3 binding +1 (Bruno, Bill, and myself)
2 non-binding +1 (Patrick and Vasiliki)
no vetoes

The vote is now closed. If anyone has objections later on,
please raise them, though!

We will proceed with a series of pull requests to implement
the framework, and we will also propose one or more small
KIPs to propose specific queries.

Thanks again,
-John

On Mon, 2021-11-22 at 12:11 -0500, Bill Bejeck wrote:
> Thanks for the well-detailed KIP, John.
> 
> It's a +1 (binding) from me.
> 
> I want to point out one thing which I think is an oversight. The "Example
> Raw Query" scan includes a line using the `kafkaStreams.serdesForStore`
> method, but it's listed in the "Rejected Alternatives" section.
> 
> Thanks,
> Bill
> 
> On Mon, Nov 22, 2021 at 9:22 AM Bruno Cadonna  wrote:
> 
> > Thanks for the KIP, John!
> > 
> > +1 (binding)
> > 
> > Best,
> > Bruno
> > 
> > On 19.11.21 18:04, Vasiliki Papavasileiou wrote:
> > > I think this KIP will greatly improve how we handle IQ in streams so +1
> > > (non-binding) from me.
> > > 
> > > Thank you John!
> > > 
> > > On Thu, Nov 18, 2021 at 9:45 PM Patrick Stuedi
> > 
> > > wrote:
> > > 
> > > > +1 (non-binding), thanks John!
> > > > -Patrick
> > > > 
> > > > On Thu, Nov 18, 2021 at 12:27 AM John Roesler 
> > wrote:
> > > > 
> > > > > Hello all,
> > > > > 
> > > > > I'd like to open the vote for KIP-796, which proposes
> > > > > a revamp of the Interactive Query APIs in Kafka Streams.
> > > > > 
> > > > > The proposal is here:
> > > > > https://cwiki.apache.org/confluence/x/34xnCw
> > > > > 
> > > > > Thanks to all who reviewed the proposal, and thanks in
> > > > > advance for taking the time to vote!
> > > > > 
> > > > > Thank you,
> > > > > -John
> > > > > 
> > > > 
> > > 
> > 



Re: [VOTE] KIP-797: Accept duplicate listener on port for IPv4/IPv6

2021-11-22 Thread Luke Chen
Hi Matthew,
Thanks for the KIP.
It makes sense to allow IPv4 and IPv6 listening on the same port for the
listener config.

+1 (non-binding)

Thank you.
Luke

On Mon, Nov 22, 2021 at 6:28 PM Matthew de Detrich
 wrote:

> Hello everyone,
>
> I would like to start a vote for KIP-797: Accept duplicate listener on port
> for IPv4/IPv6
>
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=195726330
>
> The purpose of the KIP is to loosen current validation for non advertised
> listeners so that you can have an IPv4 address and an IPv6 address on the
> same port. All other behaviour remains the same as before (since these are
> disparate IP stacks there are no technical reasons not to allow this).
>
> PR is located at https://github.com/apache/kafka/pull/11478
>
> Comments and feedback are welcome!
>
> Regards
>
> --
>
> Matthew de Detrich
>
> *Aiven Deutschland GmbH*
>
> Immanuelkirchstraße 26, 10405 Berlin
>
> Amtsgericht Charlottenburg, HRB 209739 B
>
> Geschäftsführer: Oskari Saarenmaa & Hannu Valtonen
>
> *m:* +491603708037
>
> *w:* aiven.io *e:* matthew.dedetr...@aiven.io
>


Re: Request for permission to contribute to Apache Kafka

2021-11-22 Thread Matthias J. Sax

Done.

On 11/22/21 11:33 AM, Sarah Story wrote:

Hello!

I'd like to be added to the contributors list so that I can add a KIP.

Jira ID: sarah-story
Wiki ID: sarah-story

Thank you!



Re: [DISCUSS] KIP-782: Expandable batch size in producer

2021-11-22 Thread Artem Livshits
> I think this KIP would change the behaviour of producers when there are
multiple partitions ready to be sent

This is correct, the pattern changes and becomes more coarse-grained.  But
I don't think it changes fairness over the long run.  I think it's a good
idea to change drainIndex to be random rather than round robin to avoid
forming patterns where some partitions would consistently get higher
latencies than others because they wait longer for their turn.

If we really wanted to preserve the exact patterns, we could either try to
support multiple 16KB batches from one partition per request (probably
would require protocol change to change logic on the broker for duplicate
detection) or try to re-batch 16KB batches from accumulator into larger
batches during send (additional computations) or try to consider all
partitions assigned to a broker to check if a new batch needs to be created
(i.e. compare cumulative batch size from all partitions assigned to a
broker and create new batch when cumulative size is 1MB, more complex).

Overall, it seems like just increasing the max batch size is a simpler
solution and it does favor larger batch sizes, which is beneficial not just
for production.

> ready batches well past when they ought to be sent (according to their
linger.ms)

The trigger for marking batches ready to be sent isn't changed - a batch is
ready to be sent once it reaches 16KB, so by the time larger batches start
forming, linger.ms wouldn't matter much because the batching goal is met
and the batch can be sent immediately.  Larger batches start forming once
the client starts waiting for the server, in which case some data will wait
its turn to be sent.  This will happen for some data regardless of how we
pick data to send, the question is just whether we'd have some scenarios
where some partitions would consistently experience higher latency than
others.  I think picking drainIndex randomly would prevent such scenarios.

-Artem

On Mon, Nov 22, 2021 at 2:28 AM Tom Bentley  wrote:

> Hi Luke,
>
> Thanks for the KIP!
>
> Currently because buffers are allocated using batch.size it means we can
> handle records that are that large (e.g. one big record per batch). Doesn't
> the introduction of smaller buffer sizes (batch.initial.size) mean a
> corresponding decrease in the maximum record size that the producer can
> handle? That might not be a problem if the user knows their maximum record
> size and has tuned batch.initial.size accordingly, but if the default for
> batch.initial.size < batch.size it could cause regressions for existing
> users with a large record size, I think. It should be enough for
> batch.initial.size to default to batch.size, allowing users who care about
> the memory saving in the off-peak throughput case to do the tuning, but not
> causing a regression for existing users.
>
> I think this KIP would change the behaviour of producers when there are
> multiple partitions ready to be sent: By sending all the ready buffers
> (which may now be > batch.size) for the first partition, we could end up
> excluding ready buffers for other partitions from the current send. In
> other words, as I understand the KIP currently, there's a change in
> fairness. I think the code in RecordAccumulator#drainBatchesForOneNode will
> ensure fairness in the long run, because the drainIndex will ensure that
> those other partitions each get their turn at being the first. But isn't
> there the risk that drainBatchesForOneNode would end up not sending ready
> batches well past when they ought to be sent (according to their linger.ms
> ),
> because it's sending buffers for earlier partitions too aggressively? Or,
> to put it another way, perhaps the RecordAccumulator should round-robin the
> ready buffers for _all_ the partitions before trying to fill the remaining
> space with the extra buffers (beyond the batch.size limit) for the first
> partitions?
>
> Kind regards,
>
> Tom
>
> On Wed, Oct 20, 2021 at 1:35 PM Luke Chen  wrote:
>
> > Hi Ismael and all devs,
> > Is there any comments/suggestions to this KIP?
> > If no, I'm going to update the KIP based on my previous mail, and start a
> > vote tomorrow or next week.
> >
> > Thank you.
> > Luke
> >
> > On Mon, Oct 18, 2021 at 2:40 PM Luke Chen  wrote:
> >
> > > Hi Ismael,
> > > Thanks for your comments.
> > >
> > > 1. Why do we have to reallocate the buffer? We can keep a list of
> buffers
> > > instead and avoid reallocation.
> > > -> Do you mean we allocate multiple buffers with "buffer.initial.size",
> > > and link them together (with linked list)?
> > > ex:
> > > a. We allocate 4KB initial buffer
> > > | 4KB |
> > >
> > > b. when new records reached and the remaining buffer is not enough for
> > the
> > > records, we create another batch with "batch.initial.size" buffer
> > > ex: we already have 3KB of data in the 1st buffer, and here comes the
> 2KB
> > > record
> > >
> > > | 4KB (1KB remaining) |
> > > now, record: 2KB coming
> > > We fill the 1st 

Re: [DISCUSS] Apache Kafka 3.1.0 release

2021-11-22 Thread Chris Egerton
Hi David,

I have another blocker to propose. KAFKA-13472 (
https://issues.apache.org/jira/browse/KAFKA-13472) is another regression in
Connect caused by recently-merged changes for KAFKA-12487 (
https://issues.apache.org/jira/browse/KAFKA-12487) which can lead to data
loss in sink connectors in some rare edge cases. I've opened a fix PR (
https://github.com/apache/kafka/pull/11526) already, and have also opened a
fix PR (https://github.com/apache/kafka/pull/11524) for the aforementioned
KAFKA-13469.

Please let me know if we can merge a fix for this in time for the 3.1.0
release; if not, as with KAFKA-13469, we may want to revert the changes for
the PR that cause this issue (in this case, that'd be the PR for
KAFKA-12487).

Cheers,

Chris

On Mon, Nov 22, 2021 at 11:42 AM Chris Egerton  wrote:

> Hi David,
>
> I'd like to propose KAFKA-13469 (
> https://issues.apache.org/jira/browse/KAFKA-13469) as a blocker. It is a
> regression in Connect caused by recently-merged changes for KAFKA-12226 (
> https://issues.apache.org/jira/browse/KAFKA-12226) which leads to
> duplicate records for source tasks. I plan to have a fix PR opened by the
> end of the day.
>
> Please let me know if we can merge a fix for this in time for the 3.1.0
> release; if not, we may want to revert the changes for KAFKA-12226.
>
> Cheers,
>
> Chris
>
> On Mon, Nov 15, 2021 at 5:02 AM David Jacot 
> wrote:
>
>> Hi folks,
>>
>> We reached the code freeze for the Apache Kafka 3.1 release on Friday.
>> Therefore,
>> we will only accept blockers from now on.
>>
>> There already are a couple of blockers identified which were not
>> completed before
>> the code freeze. Please, raise any new blockers to this thread.
>>
>> For all the non-blocker issues targeting 3.1.0, I will move them to
>> the next release.
>>
>> Cheers,
>> David
>>
>> On Fri, Oct 29, 2021 at 12:20 PM Dongjin Lee  wrote:
>> >
>> > Hi David,
>> >
>> > Please update the components of the following KIPs:
>> >
>> > - KIP-390: Support Compression Level - Core, Clients
>> > - KIP-653: Upgrade log4j to log4j2 - Clients, Connect, Core, Streams
>> (that
>> > is, Log4j-appender, Tools, and Trogdor are excluded.)
>> >
>> > Best,
>> > Dongjin
>> >
>> > On Fri, Oct 29, 2021 at 2:24 AM Chris Egerton
>> 
>> > wrote:
>> >
>> > > Hi David,
>> > >
>> > > I've moved KIP-618 to the "postponed" section as it will not be
>> merged in
>> > > time due to lack of review.
>> > >
>> > > Cheers,
>> > >
>> > > Chris
>> > >
>> > > On Thu, Oct 28, 2021 at 1:07 PM David Jacot
>> 
>> > > wrote:
>> > >
>> > > > Hi team,
>> > > >
>> > > > Just a quick reminder that the Feature freeze is tomorrow (October
>> 29th).
>> > > > In order to be fair with everyone in all the time zones, I plan to
>> cut
>> > > the
>> > > > release branch early next week.
>> > > >
>> > > > Cheers,
>> > > > David
>> > > >
>> > > > On Mon, Oct 18, 2021 at 9:56 AM David Jacot 
>> wrote:
>> > > >
>> > > > > Hi team,
>> > > > >
>> > > > > KIP freeze for the next major release of Apache Kafka was reached
>> > > > > last week.
>> > > > >
>> > > > > I have updated the release plan with all the adopted KIPs which
>> are
>> > > > > considered
>> > > > > for AK 3.1.0. Please, verify the plan and let me know if any KIP
>> should
>> > > > be
>> > > > > added
>> > > > > to or removed from the release plan.
>> > > > >
>> > > > > For the KIPs which are still in progress, please work closely
>> with your
>> > > > > reviewers
>> > > > > to make sure that they land on time for the feature freeze.
>> > > > >
>> > > > > The next milestone for the AK 3.1.0 release is the feature freeze
>> on
>> > > > > October 29th,
>> > > > > 2021.
>> > > > >
>> > > > > Cheers,
>> > > > > David
>> > > > >
>> > > > > On Fri, Oct 15, 2021 at 9:05 AM David Jacot 
>> > > wrote:
>> > > > >
>> > > > >> Hi folks,
>> > > > >>
>> > > > >> Just a quick reminder that the KIP freeze is today. Don't forget
>> to
>> > > > close
>> > > > >> your ongoing votes.
>> > > > >>
>> > > > >> Best,
>> > > > >> David
>> > > > >>
>> > > > >> On Thu, Oct 14, 2021 at 5:31 PM David Jacot > >
>> > > > wrote:
>> > > > >>
>> > > > >>> Hi Luke,
>> > > > >>>
>> > > > >>> Added it to the plan.
>> > > > >>>
>> > > > >>> Thanks,
>> > > > >>> David
>> > > > >>>
>> > > > >>> On Thu, Oct 14, 2021 at 10:09 AM Luke Chen 
>> > > wrote:
>> > > > >>>
>> > > >  Hi David,
>> > > >  KIP-766 is merged into trunk. Please help add it into the
>> release
>> > > > plan.
>> > > > 
>> > > >  Thank you.
>> > > >  Luke
>> > > > 
>> > > >  On Mon, Oct 11, 2021 at 10:50 PM David Jacot
>> > > >  
>> > > >  wrote:
>> > > > 
>> > > >  > Hi Michael,
>> > > >  >
>> > > >  > Sure. I have updated the release plan to include it. Thanks
>> for
>> > > the
>> > > >  > heads up.
>> > > >  >
>> > > >  > Best,
>> > > >  > David
>> > > >  >
>> > > >  > On Mon, Oct 11, 2021 at 4:39 PM Mickael Maison <
>> > > >  mickael.mai...@gmail.com>
>> > > >  > wrote:
>> > > > 

[jira] [Created] (KAFKA-13472) Connect can lose track of last committed offsets for topic partitions after partial consumer revocation

2021-11-22 Thread Chris Egerton (Jira)
Chris Egerton created KAFKA-13472:
-

 Summary: Connect can lose track of last committed offsets for 
topic partitions after partial consumer revocation
 Key: KAFKA-13472
 URL: https://issues.apache.org/jira/browse/KAFKA-13472
 Project: Kafka
  Issue Type: Bug
  Components: KafkaConnect
Affects Versions: 3.1.0, 3.0.1
Reporter: Chris Egerton
Assignee: Chris Egerton


The Connect framework tracks the last successfully-committed offsets for each 
topic partition that is currently assigned to the consumer of each sink task. 
If a sink task throws an exception from {{{}SinkTask::preCommit{}}}, the 
consumer is "rewound" by seeking to those last successfully-committed offsets 
for each topic partition, so that the same records can be redelivered to the 
task again.

With the changes from KAFKA-12487, we failed to correctly update the logic for 
tracking these last-committed offsets which can cause topic partitions to be 
missing from them after partial revocation of topic partitions from the 
consumer. Specifically, we make the assumption that, whenever an offset commit 
succeeds, the offsets that were successfully committed constitute the entirely 
of the last-committed offsets for the task; when a partial consumer revocation 
takes place, we only commit offsets for some of the topic partitions assigned 
to the task's producer, and this assumption fails.



--
This message was sent by Atlassian Jira
(v8.20.1#820001)


[jira] [Created] (KAFKA-13471) Test rolling change of KRaft controller endpoints

2021-11-22 Thread Colin McCabe (Jira)
Colin McCabe created KAFKA-13471:


 Summary: Test rolling change of KRaft controller endpoints
 Key: KAFKA-13471
 URL: https://issues.apache.org/jira/browse/KAFKA-13471
 Project: Kafka
  Issue Type: Improvement
Reporter: Colin McCabe


We should have a test of making a rolling change to KRaft controller endpoints. 
For example, going from PLAINTEXT or SSL. (This would involve going through an 
intermediate stage where the controllers exposed both endpoints.)



--
This message was sent by Atlassian Jira
(v8.20.1#820001)


Request for permission to contribute to Apache Kafka

2021-11-22 Thread Sarah Story
Hello!

I'd like to be added to the contributors list so that I can add a KIP.

Jira ID: sarah-story
Wiki ID: sarah-story

Thank you!


[jira] [Created] (KAFKA-13470) Fix the handling of secure controller ports

2021-11-22 Thread Colin McCabe (Jira)
Colin McCabe created KAFKA-13470:


 Summary: Fix the handling of secure controller ports
 Key: KAFKA-13470
 URL: https://issues.apache.org/jira/browse/KAFKA-13470
 Project: Kafka
  Issue Type: Bug
Reporter: Colin McCabe


It should be possible to configure the controller to use secure ports. 
Unfortunately, there are a few things about this that are currently broken:

* ControllerChannelManager.scala is looking at "control plane listeners" rather 
than "controller listeners" when searching for port security configurations. 
(KRaft mode actually does not support control plane listeners!)

* There is some confusion about whether to include the controller listener in 
the `listeners` configuration, on broker-only nodes. We should clarify this 
point, with a KIP addendum if needed.

* We need a junit integration test for this



--
This message was sent by Atlassian Jira
(v8.20.1#820001)


Re: [VOTE] KIP-798 Add possibility to write kafka headers in Kafka Console Producer

2021-11-22 Thread Florin Akermann
Hi Bill and David,

Thank you both for the vote.
@David: KIP is updated.

Florin

On Mon, 22 Nov 2021 at 18:28, David Jacot 
wrote:

> Hi Florin,
>
> Thanks for the KIP. I am +1 (binding).
>
> There is a small typo in the Proposed Changes section:
> `parse.header` should be `parse.headers`.
>
> Best,
> David
>
> On Mon, Nov 22, 2021 at 6:20 PM Bill Bejeck  wrote:
> >
> > Hi Florin,
> >
> > Thanks for the KIP, this seems like a very useful addition.
> >
> > +1(binding).
> >
> > -Bill
> >
> > On Mon, Nov 22, 2021 at 12:00 PM Florin Akermann <
> florin.akerm...@gmail.com>
> > wrote:
> >
> > > Hi Luke and Tom
> > >
> > > @Tom: Thanks for the vote.
> > >
> > > @Luke: Thanks for the feedback.
> > >
> > > I have updated the KIP accordingly with regards to your comments on the
> > > remaining case (false,false) and the motivation.
> > >
> > > Regarding the "not only UTF-8": As far as I understand John it is fine
> to
> > > limit the scope for this change to UTF-8 only as it is a handy
> addition on
> > > its own. Other formats can be relatively easily supported by adding
> more
> > > properties in later KIPs. In my reply to John (email from 21 Nov 2021,
> > > 11:29 UTC) I also added an explanation why I limited the scope to UTF-8
> > > only.
> > >
> > > Thanks,
> > > Florin
> > >
> > >
> > >
> > > On Mon, 22 Nov 2021 at 10:32, Tom Bentley  wrote:
> > >
> > > > Hi Florin,
> > > >
> > > > Thanks for the KIP!
> > > >
> > > > +1 (binding),
> > > >
> > > > Kind regards,
> > > >
> > > > Tom
> > > >
> > > > On Mon, Nov 22, 2021 at 6:51 AM Luke Chen  wrote:
> > > >
> > > > > Hi Florin,
> > > > > Thanks for the KIP.
> > > > >
> > > > > This KIP makes sense to me. Just a comment that the motivation
> section
> > > is
> > > > > not clearly explain why this KIP is important.
> > > > > I think John already mentioned a good motivation, which is to
> support
> > > > "not
> > > > > only UTF-8".
> > > > > You should put that into the KIP, and of course if you have other
> > > > thoughts,
> > > > > please also add them into KIP.
> > > > >
> > > > > Also, in the "public interface" section, there are 3 "Default
> parsing
> > > > > pattern", I think you should add 1 remaining case (false, false) to
> > > make
> > > > it
> > > > > complete.
> > > > >
> > > > > Otherwise, look good to me.
> > > > >
> > > > > Thank you.
> > > > > Luke
> > > > >
> > > > >
> > > > > On Sun, Nov 21, 2021 at 7:37 PM Florin Akermann <
> > > > florin.akerm...@gmail.com
> > > > > >
> > > > > wrote:
> > > > >
> > > > > > Hi John,
> > > > > >
> > > > > > Thanks for the vote and feedback.
> > > > > >
> > > > > > The thought occurred to me too.
> > > > > >
> > > > > > Do I understand it correctly: the current version of the
> > > > > > kafka-console-producer cannot be used for anything other than
> UTF-8
> > > > keys
> > > > > > and values?
> > > > > > (There is no other implementation of MessageReader other than the
> > > > > > ConsoleProducer$LineMessageReader)
> > > > > > In other words, currently users seem to only apply it with utf-8
> > > > strings
> > > > > > for keys and values?
> > > > > > This is why I figured I would not deviate from this assumption
> solely
> > > > for
> > > > > > the headers.
> > > > > >
> > > > > > I will happily raise another KIP / Jira if there is a need to
> specify
> > > > > other
> > > > > > formats / serializers for headers, keys and/or values.
> > > > > >
> > > > > > Thanks,
> > > > > > Florin
> > > > > >
> > > > > >
> > > > > > On Sat, 20 Nov 2021 at 19:34, John Roesler 
> > > > wrote:
> > > > > >
> > > > > > > Hi Florin,
> > > > > > >
> > > > > > > Thanks for the KIP!
> > > > > > >
> > > > > > > I think the assumption that header values are UTF-8 strings
> might
> > > not
> > > > > > hold
> > > > > > > up in the long run, but it seems like we can easily add a
> property
> > > > > later
> > > > > > to
> > > > > > > specify the format. It seems like this scope is probably a
> handy
> > > > > addition
> > > > > > > on its own.
> > > > > > >
> > > > > > > I’m +1 (binding)
> > > > > > >
> > > > > > > Thanks,
> > > > > > > John
> > > > > > >
> > > > > > >
> > > > > > > On Fri, Nov 19, 2021, at 15:06, flo wrote:
> > > > > > > > <
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-798%3A+Add+possibility+to+write+kafka+headers+in+Kafka+Console+Producer
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
>


Re: [VOTE] KIP-798 Add possibility to write kafka headers in Kafka Console Producer

2021-11-22 Thread David Jacot
Hi Florin,

Thanks for the KIP. I am +1 (binding).

There is a small typo in the Proposed Changes section:
`parse.header` should be `parse.headers`.

Best,
David

On Mon, Nov 22, 2021 at 6:20 PM Bill Bejeck  wrote:
>
> Hi Florin,
>
> Thanks for the KIP, this seems like a very useful addition.
>
> +1(binding).
>
> -Bill
>
> On Mon, Nov 22, 2021 at 12:00 PM Florin Akermann 
> wrote:
>
> > Hi Luke and Tom
> >
> > @Tom: Thanks for the vote.
> >
> > @Luke: Thanks for the feedback.
> >
> > I have updated the KIP accordingly with regards to your comments on the
> > remaining case (false,false) and the motivation.
> >
> > Regarding the "not only UTF-8": As far as I understand John it is fine to
> > limit the scope for this change to UTF-8 only as it is a handy addition on
> > its own. Other formats can be relatively easily supported by adding more
> > properties in later KIPs. In my reply to John (email from 21 Nov 2021,
> > 11:29 UTC) I also added an explanation why I limited the scope to UTF-8
> > only.
> >
> > Thanks,
> > Florin
> >
> >
> >
> > On Mon, 22 Nov 2021 at 10:32, Tom Bentley  wrote:
> >
> > > Hi Florin,
> > >
> > > Thanks for the KIP!
> > >
> > > +1 (binding),
> > >
> > > Kind regards,
> > >
> > > Tom
> > >
> > > On Mon, Nov 22, 2021 at 6:51 AM Luke Chen  wrote:
> > >
> > > > Hi Florin,
> > > > Thanks for the KIP.
> > > >
> > > > This KIP makes sense to me. Just a comment that the motivation section
> > is
> > > > not clearly explain why this KIP is important.
> > > > I think John already mentioned a good motivation, which is to support
> > > "not
> > > > only UTF-8".
> > > > You should put that into the KIP, and of course if you have other
> > > thoughts,
> > > > please also add them into KIP.
> > > >
> > > > Also, in the "public interface" section, there are 3 "Default parsing
> > > > pattern", I think you should add 1 remaining case (false, false) to
> > make
> > > it
> > > > complete.
> > > >
> > > > Otherwise, look good to me.
> > > >
> > > > Thank you.
> > > > Luke
> > > >
> > > >
> > > > On Sun, Nov 21, 2021 at 7:37 PM Florin Akermann <
> > > florin.akerm...@gmail.com
> > > > >
> > > > wrote:
> > > >
> > > > > Hi John,
> > > > >
> > > > > Thanks for the vote and feedback.
> > > > >
> > > > > The thought occurred to me too.
> > > > >
> > > > > Do I understand it correctly: the current version of the
> > > > > kafka-console-producer cannot be used for anything other than UTF-8
> > > keys
> > > > > and values?
> > > > > (There is no other implementation of MessageReader other than the
> > > > > ConsoleProducer$LineMessageReader)
> > > > > In other words, currently users seem to only apply it with utf-8
> > > strings
> > > > > for keys and values?
> > > > > This is why I figured I would not deviate from this assumption solely
> > > for
> > > > > the headers.
> > > > >
> > > > > I will happily raise another KIP / Jira if there is a need to specify
> > > > other
> > > > > formats / serializers for headers, keys and/or values.
> > > > >
> > > > > Thanks,
> > > > > Florin
> > > > >
> > > > >
> > > > > On Sat, 20 Nov 2021 at 19:34, John Roesler 
> > > wrote:
> > > > >
> > > > > > Hi Florin,
> > > > > >
> > > > > > Thanks for the KIP!
> > > > > >
> > > > > > I think the assumption that header values are UTF-8 strings might
> > not
> > > > > hold
> > > > > > up in the long run, but it seems like we can easily add a property
> > > > later
> > > > > to
> > > > > > specify the format. It seems like this scope is probably a handy
> > > > addition
> > > > > > on its own.
> > > > > >
> > > > > > I’m +1 (binding)
> > > > > >
> > > > > > Thanks,
> > > > > > John
> > > > > >
> > > > > >
> > > > > > On Fri, Nov 19, 2021, at 15:06, flo wrote:
> > > > > > > <
> > > > > >
> > > > >
> > > >
> > >
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-798%3A+Add+possibility+to+write+kafka+headers+in+Kafka+Console+Producer
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >


Re: [VOTE] KIP-798 Add possibility to write kafka headers in Kafka Console Producer

2021-11-22 Thread Bill Bejeck
Hi Florin,

Thanks for the KIP, this seems like a very useful addition.

+1(binding).

-Bill

On Mon, Nov 22, 2021 at 12:00 PM Florin Akermann 
wrote:

> Hi Luke and Tom
>
> @Tom: Thanks for the vote.
>
> @Luke: Thanks for the feedback.
>
> I have updated the KIP accordingly with regards to your comments on the
> remaining case (false,false) and the motivation.
>
> Regarding the "not only UTF-8": As far as I understand John it is fine to
> limit the scope for this change to UTF-8 only as it is a handy addition on
> its own. Other formats can be relatively easily supported by adding more
> properties in later KIPs. In my reply to John (email from 21 Nov 2021,
> 11:29 UTC) I also added an explanation why I limited the scope to UTF-8
> only.
>
> Thanks,
> Florin
>
>
>
> On Mon, 22 Nov 2021 at 10:32, Tom Bentley  wrote:
>
> > Hi Florin,
> >
> > Thanks for the KIP!
> >
> > +1 (binding),
> >
> > Kind regards,
> >
> > Tom
> >
> > On Mon, Nov 22, 2021 at 6:51 AM Luke Chen  wrote:
> >
> > > Hi Florin,
> > > Thanks for the KIP.
> > >
> > > This KIP makes sense to me. Just a comment that the motivation section
> is
> > > not clearly explain why this KIP is important.
> > > I think John already mentioned a good motivation, which is to support
> > "not
> > > only UTF-8".
> > > You should put that into the KIP, and of course if you have other
> > thoughts,
> > > please also add them into KIP.
> > >
> > > Also, in the "public interface" section, there are 3 "Default parsing
> > > pattern", I think you should add 1 remaining case (false, false) to
> make
> > it
> > > complete.
> > >
> > > Otherwise, look good to me.
> > >
> > > Thank you.
> > > Luke
> > >
> > >
> > > On Sun, Nov 21, 2021 at 7:37 PM Florin Akermann <
> > florin.akerm...@gmail.com
> > > >
> > > wrote:
> > >
> > > > Hi John,
> > > >
> > > > Thanks for the vote and feedback.
> > > >
> > > > The thought occurred to me too.
> > > >
> > > > Do I understand it correctly: the current version of the
> > > > kafka-console-producer cannot be used for anything other than UTF-8
> > keys
> > > > and values?
> > > > (There is no other implementation of MessageReader other than the
> > > > ConsoleProducer$LineMessageReader)
> > > > In other words, currently users seem to only apply it with utf-8
> > strings
> > > > for keys and values?
> > > > This is why I figured I would not deviate from this assumption solely
> > for
> > > > the headers.
> > > >
> > > > I will happily raise another KIP / Jira if there is a need to specify
> > > other
> > > > formats / serializers for headers, keys and/or values.
> > > >
> > > > Thanks,
> > > > Florin
> > > >
> > > >
> > > > On Sat, 20 Nov 2021 at 19:34, John Roesler 
> > wrote:
> > > >
> > > > > Hi Florin,
> > > > >
> > > > > Thanks for the KIP!
> > > > >
> > > > > I think the assumption that header values are UTF-8 strings might
> not
> > > > hold
> > > > > up in the long run, but it seems like we can easily add a property
> > > later
> > > > to
> > > > > specify the format. It seems like this scope is probably a handy
> > > addition
> > > > > on its own.
> > > > >
> > > > > I’m +1 (binding)
> > > > >
> > > > > Thanks,
> > > > > John
> > > > >
> > > > >
> > > > > On Fri, Nov 19, 2021, at 15:06, flo wrote:
> > > > > > <
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-798%3A+Add+possibility+to+write+kafka+headers+in+Kafka+Console+Producer
> > > > > >
> > > > >
> > > >
> > >
> >
>


Re: [VOTE} KIP-796: Interactive Query v2

2021-11-22 Thread Bill Bejeck
Thanks for the well-detailed KIP, John.

It's a +1 (binding) from me.

I want to point out one thing which I think is an oversight. The "Example
Raw Query" scan includes a line using the `kafkaStreams.serdesForStore`
method, but it's listed in the "Rejected Alternatives" section.

Thanks,
Bill

On Mon, Nov 22, 2021 at 9:22 AM Bruno Cadonna  wrote:

> Thanks for the KIP, John!
>
> +1 (binding)
>
> Best,
> Bruno
>
> On 19.11.21 18:04, Vasiliki Papavasileiou wrote:
> > I think this KIP will greatly improve how we handle IQ in streams so +1
> > (non-binding) from me.
> >
> > Thank you John!
> >
> > On Thu, Nov 18, 2021 at 9:45 PM Patrick Stuedi
> 
> > wrote:
> >
> >> +1 (non-binding), thanks John!
> >> -Patrick
> >>
> >> On Thu, Nov 18, 2021 at 12:27 AM John Roesler 
> wrote:
> >>
> >>> Hello all,
> >>>
> >>> I'd like to open the vote for KIP-796, which proposes
> >>> a revamp of the Interactive Query APIs in Kafka Streams.
> >>>
> >>> The proposal is here:
> >>> https://cwiki.apache.org/confluence/x/34xnCw
> >>>
> >>> Thanks to all who reviewed the proposal, and thanks in
> >>> advance for taking the time to vote!
> >>>
> >>> Thank you,
> >>> -John
> >>>
> >>
> >
>


Re: [Vote] KIP-787 - MM2 Interface to manage Kafka resources Kafka/KIPs

2021-11-22 Thread Omnia Ibrahim
Hi All,

Just thought of bumping this voting thread again to see if we can form a
consensus around this.

Thanks


On Tue, Nov 16, 2021 at 2:16 PM Omnia Ibrahim 
wrote:

> Hi,
> I'd like to start a vote on KIP-787
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-787%3A+MM2+Interface+to+manage+Kafka+resources
> 
>
> Thanks
> Omnia
>


Re: [VOTE] KIP-798 Add possibility to write kafka headers in Kafka Console Producer

2021-11-22 Thread Florin Akermann
Hi Luke and Tom

@Tom: Thanks for the vote.

@Luke: Thanks for the feedback.

I have updated the KIP accordingly with regards to your comments on the
remaining case (false,false) and the motivation.

Regarding the "not only UTF-8": As far as I understand John it is fine to
limit the scope for this change to UTF-8 only as it is a handy addition on
its own. Other formats can be relatively easily supported by adding more
properties in later KIPs. In my reply to John (email from 21 Nov 2021,
11:29 UTC) I also added an explanation why I limited the scope to UTF-8
only.

Thanks,
Florin



On Mon, 22 Nov 2021 at 10:32, Tom Bentley  wrote:

> Hi Florin,
>
> Thanks for the KIP!
>
> +1 (binding),
>
> Kind regards,
>
> Tom
>
> On Mon, Nov 22, 2021 at 6:51 AM Luke Chen  wrote:
>
> > Hi Florin,
> > Thanks for the KIP.
> >
> > This KIP makes sense to me. Just a comment that the motivation section is
> > not clearly explain why this KIP is important.
> > I think John already mentioned a good motivation, which is to support
> "not
> > only UTF-8".
> > You should put that into the KIP, and of course if you have other
> thoughts,
> > please also add them into KIP.
> >
> > Also, in the "public interface" section, there are 3 "Default parsing
> > pattern", I think you should add 1 remaining case (false, false) to make
> it
> > complete.
> >
> > Otherwise, look good to me.
> >
> > Thank you.
> > Luke
> >
> >
> > On Sun, Nov 21, 2021 at 7:37 PM Florin Akermann <
> florin.akerm...@gmail.com
> > >
> > wrote:
> >
> > > Hi John,
> > >
> > > Thanks for the vote and feedback.
> > >
> > > The thought occurred to me too.
> > >
> > > Do I understand it correctly: the current version of the
> > > kafka-console-producer cannot be used for anything other than UTF-8
> keys
> > > and values?
> > > (There is no other implementation of MessageReader other than the
> > > ConsoleProducer$LineMessageReader)
> > > In other words, currently users seem to only apply it with utf-8
> strings
> > > for keys and values?
> > > This is why I figured I would not deviate from this assumption solely
> for
> > > the headers.
> > >
> > > I will happily raise another KIP / Jira if there is a need to specify
> > other
> > > formats / serializers for headers, keys and/or values.
> > >
> > > Thanks,
> > > Florin
> > >
> > >
> > > On Sat, 20 Nov 2021 at 19:34, John Roesler 
> wrote:
> > >
> > > > Hi Florin,
> > > >
> > > > Thanks for the KIP!
> > > >
> > > > I think the assumption that header values are UTF-8 strings might not
> > > hold
> > > > up in the long run, but it seems like we can easily add a property
> > later
> > > to
> > > > specify the format. It seems like this scope is probably a handy
> > addition
> > > > on its own.
> > > >
> > > > I’m +1 (binding)
> > > >
> > > > Thanks,
> > > > John
> > > >
> > > >
> > > > On Fri, Nov 19, 2021, at 15:06, flo wrote:
> > > > > <
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-798%3A+Add+possibility+to+write+kafka+headers+in+Kafka+Console+Producer
> > > > >
> > > >
> > >
> >
>


Re: [DISCUSS] Apache Kafka 3.1.0 release

2021-11-22 Thread Chris Egerton
Hi David,

I'd like to propose KAFKA-13469 (
https://issues.apache.org/jira/browse/KAFKA-13469) as a blocker. It is a
regression in Connect caused by recently-merged changes for KAFKA-12226 (
https://issues.apache.org/jira/browse/KAFKA-12226) which leads to duplicate
records for source tasks. I plan to have a fix PR opened by the end of the
day.

Please let me know if we can merge a fix for this in time for the 3.1.0
release; if not, we may want to revert the changes for KAFKA-12226.

Cheers,

Chris

On Mon, Nov 15, 2021 at 5:02 AM David Jacot 
wrote:

> Hi folks,
>
> We reached the code freeze for the Apache Kafka 3.1 release on Friday.
> Therefore,
> we will only accept blockers from now on.
>
> There already are a couple of blockers identified which were not
> completed before
> the code freeze. Please, raise any new blockers to this thread.
>
> For all the non-blocker issues targeting 3.1.0, I will move them to
> the next release.
>
> Cheers,
> David
>
> On Fri, Oct 29, 2021 at 12:20 PM Dongjin Lee  wrote:
> >
> > Hi David,
> >
> > Please update the components of the following KIPs:
> >
> > - KIP-390: Support Compression Level - Core, Clients
> > - KIP-653: Upgrade log4j to log4j2 - Clients, Connect, Core, Streams
> (that
> > is, Log4j-appender, Tools, and Trogdor are excluded.)
> >
> > Best,
> > Dongjin
> >
> > On Fri, Oct 29, 2021 at 2:24 AM Chris Egerton
> 
> > wrote:
> >
> > > Hi David,
> > >
> > > I've moved KIP-618 to the "postponed" section as it will not be merged
> in
> > > time due to lack of review.
> > >
> > > Cheers,
> > >
> > > Chris
> > >
> > > On Thu, Oct 28, 2021 at 1:07 PM David Jacot
> 
> > > wrote:
> > >
> > > > Hi team,
> > > >
> > > > Just a quick reminder that the Feature freeze is tomorrow (October
> 29th).
> > > > In order to be fair with everyone in all the time zones, I plan to
> cut
> > > the
> > > > release branch early next week.
> > > >
> > > > Cheers,
> > > > David
> > > >
> > > > On Mon, Oct 18, 2021 at 9:56 AM David Jacot 
> wrote:
> > > >
> > > > > Hi team,
> > > > >
> > > > > KIP freeze for the next major release of Apache Kafka was reached
> > > > > last week.
> > > > >
> > > > > I have updated the release plan with all the adopted KIPs which are
> > > > > considered
> > > > > for AK 3.1.0. Please, verify the plan and let me know if any KIP
> should
> > > > be
> > > > > added
> > > > > to or removed from the release plan.
> > > > >
> > > > > For the KIPs which are still in progress, please work closely with
> your
> > > > > reviewers
> > > > > to make sure that they land on time for the feature freeze.
> > > > >
> > > > > The next milestone for the AK 3.1.0 release is the feature freeze
> on
> > > > > October 29th,
> > > > > 2021.
> > > > >
> > > > > Cheers,
> > > > > David
> > > > >
> > > > > On Fri, Oct 15, 2021 at 9:05 AM David Jacot 
> > > wrote:
> > > > >
> > > > >> Hi folks,
> > > > >>
> > > > >> Just a quick reminder that the KIP freeze is today. Don't forget
> to
> > > > close
> > > > >> your ongoing votes.
> > > > >>
> > > > >> Best,
> > > > >> David
> > > > >>
> > > > >> On Thu, Oct 14, 2021 at 5:31 PM David Jacot 
> > > > wrote:
> > > > >>
> > > > >>> Hi Luke,
> > > > >>>
> > > > >>> Added it to the plan.
> > > > >>>
> > > > >>> Thanks,
> > > > >>> David
> > > > >>>
> > > > >>> On Thu, Oct 14, 2021 at 10:09 AM Luke Chen 
> > > wrote:
> > > > >>>
> > > >  Hi David,
> > > >  KIP-766 is merged into trunk. Please help add it into the
> release
> > > > plan.
> > > > 
> > > >  Thank you.
> > > >  Luke
> > > > 
> > > >  On Mon, Oct 11, 2021 at 10:50 PM David Jacot
> > > >  
> > > >  wrote:
> > > > 
> > > >  > Hi Michael,
> > > >  >
> > > >  > Sure. I have updated the release plan to include it. Thanks
> for
> > > the
> > > >  > heads up.
> > > >  >
> > > >  > Best,
> > > >  > David
> > > >  >
> > > >  > On Mon, Oct 11, 2021 at 4:39 PM Mickael Maison <
> > > >  mickael.mai...@gmail.com>
> > > >  > wrote:
> > > >  >
> > > >  > > Hi David,
> > > >  > >
> > > >  > > You can add KIP-690 to the release plan. The vote passed
> months
> > > > ago
> > > >  > > and I merged the PR today.
> > > >  > >
> > > >  > > Thanks
> > > >  > >
> > > >  > > On Fri, Oct 8, 2021 at 8:32 AM David Jacot
> > > >  
> > > >  > > wrote:
> > > >  > > >
> > > >  > > > Hi folks,
> > > >  > > >
> > > >  > > > Just a quick reminder that KIP Freeze is next Friday,
> October
> > > >  15th.
> > > >  > > >
> > > >  > > > Cheers,
> > > >  > > > David
> > > >  > > >
> > > >  > > > On Wed, Sep 29, 2021 at 3:52 PM Chris Egerton
> > > >  > > 
> > > >  > > > wrote:
> > > >  > > >
> > > >  > > > > Thanks David!
> > > >  > > > >
> > > >  > > > > On Wed, Sep 29, 2021 at 2:56 AM David Jacot
> > > >  > > 
> > > >  > > > > wrote:
> > > >  > > > >
> > > >  > > > > > Hi Chris,
> > > >  > > > > >
> > > > 

[jira] [Created] (KAFKA-13469) End-of-life offset commit for source task can take place before all records are flushed

2021-11-22 Thread Chris Egerton (Jira)
Chris Egerton created KAFKA-13469:
-

 Summary: End-of-life offset commit for source task can take place 
before all records are flushed
 Key: KAFKA-13469
 URL: https://issues.apache.org/jira/browse/KAFKA-13469
 Project: Kafka
  Issue Type: Bug
  Components: KafkaConnect
Affects Versions: 3.1.0, 3.0.1
Reporter: Chris Egerton
Assignee: Chris Egerton
 Fix For: 3.1.0, 3.0.1


When we fixed KAFKA-12226, we made offset commits for source tasks take place 
without blocking for any in-flight records to be acknowledged. While a task is 
running, this change should yield significant benefits in some cases and allow 
us to continue to commit offsets even when a topic partition on the broker is 
unavailable or the producer is unable to send records to Kafka as quickly as 
they are produced by the task.

However, this becomes problematic when a task is scheduled for shutdown with 
in-flight records. During shutdown, the latest committable offsets are 
calculated, and then flushed to the offset backing store (in distributed mode, 
this is the offsets topic). During that flush, the task's producer may continue 
to send records to Kafka, but their offsets will not be committed, which causes 
these records to be redelivered if/when the task is restarted.

Essentially, duplicate records are now possible even in healthy source tasks.



--
This message was sent by Atlassian Jira
(v8.20.1#820001)


Re: [VOTE} KIP-796: Interactive Query v2

2021-11-22 Thread Bruno Cadonna

Thanks for the KIP, John!

+1 (binding)

Best,
Bruno

On 19.11.21 18:04, Vasiliki Papavasileiou wrote:

I think this KIP will greatly improve how we handle IQ in streams so +1
(non-binding) from me.

Thank you John!

On Thu, Nov 18, 2021 at 9:45 PM Patrick Stuedi 
wrote:


+1 (non-binding), thanks John!
-Patrick

On Thu, Nov 18, 2021 at 12:27 AM John Roesler  wrote:


Hello all,

I'd like to open the vote for KIP-796, which proposes
a revamp of the Interactive Query APIs in Kafka Streams.

The proposal is here:
https://cwiki.apache.org/confluence/x/34xnCw

Thanks to all who reviewed the proposal, and thanks in
advance for taking the time to vote!

Thank you,
-John







[jira] [Resolved] (KAFKA-13455) The Apache Kafka quickstart guide does not contain any steps for running Kafka Connect

2021-11-22 Thread Mickael Maison (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-13455?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Mickael Maison resolved KAFKA-13455.

Fix Version/s: 3.2.0
   Resolution: Fixed

> The Apache Kafka quickstart guide does not contain any steps for running 
> Kafka Connect
> --
>
> Key: KAFKA-13455
> URL: https://issues.apache.org/jira/browse/KAFKA-13455
> Project: Kafka
>  Issue Type: Improvement
>  Components: documentation
>Affects Versions: 2.7.2, 2.8.1, 3.0.0
>Reporter: Kate Stanley
>Assignee: Kate Stanley
>Priority: Major
> Fix For: 3.2.0
>
>
> The Apache Kafka quickstart guide does not contain any steps for running 
> Kafka Connect. Instead it links to the User guide, which just links back to 
> the quickstart. The steps are present in version 2.6 of the documentation, 
> but not in the latest.
> See [https://kafka.apache.org/26/documentation/#quickstart_kafkaconnect] vs 
> https://kafka.apache.org/documentation/#quickstart_kafkaconnect
>  



--
This message was sent by Atlassian Jira
(v8.20.1#820001)


Build failed in Jenkins: Kafka » kafka-2.6-jdk8 #143

2021-11-22 Thread Apache Jenkins Server
See 


Changes:

[github] MINOR: Bump 2.6 to use 2.6.3 (#11495)


--
[...truncated 3.18 MB...]

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldNotAllowToCreateTopicWithNullTopicNameWithNullKey PASSED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldCreateNullKeyConsumerRecordWithOtherTopicNameAndTimestampWithTimetamp 
STARTED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldCreateNullKeyConsumerRecordWithOtherTopicNameAndTimestampWithTimetamp 
PASSED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldCreateConsumerRecordWithTimestamp STARTED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldCreateConsumerRecordWithTimestamp PASSED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldNotAllowToCreateTopicWithNullHeaders STARTED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldNotAllowToCreateTopicWithNullHeaders PASSED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldNotAllowToCreateTopicWithNullTopicNameWithDefaultTimestamp STARTED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldNotAllowToCreateTopicWithNullTopicNameWithDefaultTimestamp PASSED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldRequireCustomTopicNameIfNotDefaultFactoryTopicName STARTED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldRequireCustomTopicNameIfNotDefaultFactoryTopicName PASSED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldRequireCustomTopicNameIfNotDefaultFactoryTopicNameWithNullKey STARTED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldRequireCustomTopicNameIfNotDefaultFactoryTopicNameWithNullKey PASSED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldCreateConsumerRecord STARTED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldCreateConsumerRecord PASSED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldCreateNullKeyConsumerRecord STARTED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldCreateNullKeyConsumerRecord PASSED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldCreateConsumerRecordWithOtherTopicName STARTED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldCreateConsumerRecordWithOtherTopicName PASSED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > shouldAdvanceTime 
STARTED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > shouldAdvanceTime 
PASSED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldNotAllowToCreateTopicWithNullTopicNameWithKeyValuePairs STARTED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldNotAllowToCreateTopicWithNullTopicNameWithKeyValuePairs PASSED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldRequireCustomTopicNameIfNotDefaultFactoryTopicNameWithKeyValuePairsAndCustomTimestamps
 STARTED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldRequireCustomTopicNameIfNotDefaultFactoryTopicNameWithKeyValuePairsAndCustomTimestamps
 PASSED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldRequireCustomTopicNameIfNotDefaultFactoryTopicNameWithKeyValuePairs 
STARTED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldRequireCustomTopicNameIfNotDefaultFactoryTopicNameWithKeyValuePairs PASSED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldCreateConsumerRecordWithOtherTopicNameAndTimestamp STARTED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldCreateConsumerRecordWithOtherTopicNameAndTimestamp PASSED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldNotAllowToCreateTopicWithNullTopicNameWithKeyValuePairsAndCustomTimestamps
 STARTED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldNotAllowToCreateTopicWithNullTopicNameWithKeyValuePairsAndCustomTimestamps
 PASSED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldNotAllowToCreateTopicWithNullTopicName STARTED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldNotAllowToCreateTopicWithNullTopicName PASSED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldCreateConsumerRecordsFromKeyValuePairsWithCustomTimestampAndIncrementsAndNotAdvanceTime
 STARTED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldCreateConsumerRecordsFromKeyValuePairsWithCustomTimestampAndIncrementsAndNotAdvanceTime
 PASSED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldCreateNullKeyConsumerRecordWithTimestampWithTimestamp STARTED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldCreateNullKeyConsumerRecordWithTimestampWithTimestamp PASSED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 

[GitHub] [kafka-site] tombentley merged pull request #385: MINOR: Add Tom Bentley as PMC member

2021-11-22 Thread GitBox


tombentley merged pull request #385:
URL: https://github.com/apache/kafka-site/pull/385


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[ANNOUNCE] Apache Kafka 2.7.2

2021-11-22 Thread Mickael Maison
The Apache Kafka community is pleased to announce the release for
Apache Kafka 2.7.2

This is a bug fix release and it includes fixes and improvements from
26 JIRAs, including a fix for CVE-2021-38153.

All of the changes in this release can be found in the release notes:
https://downloads.apache.org/kafka/2.7.2/RELEASE_NOTES.html


You can download the source and binary release (Scala 2.12 and 2.13) from:
https://kafka.apache.org/downloads#2.7.2

---


Apache Kafka is a distributed streaming platform with four core APIs:


** The Producer API allows an application to publish a stream records to
one or more Kafka topics.

** The Consumer API allows an application to subscribe to one or more
topics and process the stream of records produced to them.

** The Streams API allows an application to act as a stream processor,
consuming an input stream from one or more topics and producing an
output stream to one or more output topics, effectively transforming the
input streams to output streams.

** The Connector API allows building and running reusable producers or
consumers that connect Kafka topics to existing applications or data
systems. For example, a connector to a relational database might
capture every change to a table.


With these APIs, Kafka can be used for two broad classes of application:

** Building real-time streaming data pipelines that reliably get data
between systems or applications.

** Building real-time streaming applications that transform or react
to the streams of data.


Apache Kafka is in use at large and small companies worldwide, including
Capital One, Goldman Sachs, ING, LinkedIn, Netflix, Pinterest, Rabobank,
Target, The New York Times, Uber, Yelp, and Zalando, among others.

A big thank you for the following 20 contributors to this release!

A. Sophie Blee-Goldman, Alexander Iskuskov, Chris Egerton, David
Arthur, Davor Poldrugo, Dejan Stojadinović, Ismael Juma, Jason
Gustafson, Justine Olshan, Konstantine Karantasis, Lee Dongjin, Luke
Chen, Matthias J. Sax, Michael Carter, Mickael Maison, Rajini Sivaram,
Randall Hauch, Shay Elkin, Stanislav Vodetskyi, Tom Bentley

We welcome your help and feedback. For more information on how to
report problems, and to get involved, visit the project website at
https://kafka.apache.org/

Thank you!


Regards,
Mickael Maison


[GitHub] [kafka-site] mimaison merged pull request #383: MINOR: Updates for 2.7.2

2021-11-22 Thread GitBox


mimaison merged pull request #383:
URL: https://github.com/apache/kafka-site/pull/383


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[ANNOUNCE] Apache Kafka 2.6.3

2021-11-22 Thread Mickael Maison
The Apache Kafka community is pleased to announce the release for
Apache Kafka 2.6.3

This is a bug fix release and it includes fixes and improvements from
11 JIRAs, including a fix for CVE-2021-38153.

All of the changes in this release can be found in the release notes:
https://www.apache.org/dist/kafka/2.6.3/RELEASE_NOTES.html


You can download the source and binary release (Scala 2.12 and 2.13) from:
https://kafka.apache.org/downloads#2.6.3

---


Apache Kafka is a distributed streaming platform with four core APIs:


** The Producer API allows an application to publish a stream records to
one or more Kafka topics.

** The Consumer API allows an application to subscribe to one or more
topics and process the stream of records produced to them.

** The Streams API allows an application to act as a stream processor,
consuming an input stream from one or more topics and producing an
output stream to one or more output topics, effectively transforming the
input streams to output streams.

** The Connector API allows building and running reusable producers or
consumers that connect Kafka topics to existing applications or data
systems. For example, a connector to a relational database might
capture every change to a table.


With these APIs, Kafka can be used for two broad classes of application:

** Building real-time streaming data pipelines that reliably get data
between systems or applications.

** Building real-time streaming applications that transform or react
to the streams of data.


Apache Kafka is in use at large and small companies worldwide, including
Capital One, Goldman Sachs, ING, LinkedIn, Netflix, Pinterest, Rabobank,
Target, The New York Times, Uber, Yelp, and Zalando, among others.

A big thank you for the following 12 contributors to this release!

A. Sophie Blee-Goldman, ableegoldman, Alexander Iskuskov, Chris
Egerton, Ismael Juma, Jason Gustafson, Konstantine Karantasis, Lee
Dongjin, Matthias J. Sax, Mickael Maison, Rajini Sivaram, Randall
Hauch

We welcome your help and feedback. For more information on how to
report problems, and to get involved, visit the project website at
https://kafka.apache.org/

Thank you!


Regards,
Mickael Maison


[GitHub] [kafka-site] mimaison merged pull request #382: MINOR: Updates for 2.6.3

2021-11-22 Thread GitBox


mimaison merged pull request #382:
URL: https://github.com/apache/kafka-site/pull/382


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




Re: [DISCUSS] KIP-682: Connect TimestampConverter support for multiple fields and multiple input formats

2021-11-22 Thread Joshua Grisham
Hi Randall, thanks again for the great feedback! Sorry it took me a minute
to get back on this one--I have been a bit full up over the last few
months--but now I will try to brush it off and see if we can figure out if
it makes sense to move forward and possibly close this one out.

Regarding backward compatibility -- the intention of my proposed changes
(including the candidate PR) was that it still is backward compatible, but
I realize that what I wrote in the KIP was more theoretical about what
could happen instead of what I actually did in the PR (that I actually made
it backward-compatible).  So now I have updated the text in the KIP to
hopefully more accurately reflect this and I think/hope that it, along with
what is in the candidate PR, addresses everything you mentioned in your
first and second paragraph.

For clarity here is exactly what I have done in the PR:

First I saw that the way the config properties for TimestampConverter
lacked the same structure as it did for other SMTs (e.g. ReplaceField as
one example), especially since I was adding several new properties, so I
added the same ConfigName and ConfigDefault interfaces (instead of just
having various string attributes on the TimestampConverter class itself).
The existing public class  string attributes FIELD_CONFIG,
TARGET_TYPE_CONFIG, and FORMAT_CONFIG all still exist and all still can be
continued to used by anyone else, they will just get warnings at build time
since I marked them with @Deprecated.  I did remove the private class
strings since they were no longer used in the class but this should not
have any impact on compatibility.

Second I added the new properties to CONFIG_DEF but kept the existing
properties there (so they still exist and can be used) and in the
configure() method I have used ConfigUtils.translateDeprecatedConfigs() to
alias the old "field" property to the new "fields" property automatically
just like was done in ReplaceField from the change from KIP-629 (
https://github.com/apache/kafka/commit/eab61cad2c418786ab5e58aa1267f689d82a61d1
)
In the end it means that a user can still have their connector use the
"field" property but when the configure() method is called it will just put
whatever they set in "field" to "fields" automatically.

Also the property "format" still behaves exactly the same as before if it
is used (it continues to function as a SimpleDateFormatter input and
output) it is just that there is an additional capability added by
voluntarily adopting the two new properties "format.input" and
"format.output".

As far as I can tell, in theory someone could have an existing connector,
using TimestampConverter (or something else which uses the API even),
upgrade to this version, and not change anything else, and it still behaves
the same as before.  But perhaps it should not be marked as @Deprecated to
avoid the warnings or otherwise is there a clearly defined deprecation
policy that can be followed? I tried to search and look in the Kafka docs
but could not find anything after several attempts and rabbit-holes
unfortunately (closest I seem to find is just a list or documentation on
what has been deprecated in each release).

However please do say if there is anything else you think is missing either
in the implementation or now in the updated KIP regarding this!

Also hopefully with my updates to the KIP it has addressed your other
comments or questions, but please feel free to give it another look when
you have time and I would welcome any feedback that you have.

I also noticed there was one small change where my PR now has a merge
conflict but it should be pretty minimal to update as long as everything
else is still looking ok.. I will try to take a look at this soon as well
if I am able.

Thanks again and have a great week!

Best,
Joshua



Den ons 28 juli 2021 kl 16:05 skrev Randall Hauch :

> Thanks for the contribution, Joshua. Overall I think this is a really good
> improvement to this connector, for all the reasons cited in the motivation
> section. I do have some requests, though.
>
> The biggest issue I have is that the change is not backward compatible.
> We've worked really hard to make sure that all Connect APIs (including
> configuration properties) are always backward compatible because this makes
> it trivial for users to upgrade from one Connect release to a newer one.
> After all, any users that are already using this SMT in their connectors
> should be able to upgrade Connect and still have the SMT behave exactly as
> it did in earlier releases -- unless they opt in to changing their
> configuration to use the new feature. So I would request that the KIP and
> proposed implementation be changed to not *change* existing configuration
> properties. We can of course introduce new config properties, but they
> should have a default that results in exactly the same old behavior as in
> earlier releases. A third option is to create a new SMT, though we should
> try really hard to just 

Re: [DISCUSS] KIP-782: Expandable batch size in producer

2021-11-22 Thread Tom Bentley
Hi Luke,

Thanks for the KIP!

Currently because buffers are allocated using batch.size it means we can
handle records that are that large (e.g. one big record per batch). Doesn't
the introduction of smaller buffer sizes (batch.initial.size) mean a
corresponding decrease in the maximum record size that the producer can
handle? That might not be a problem if the user knows their maximum record
size and has tuned batch.initial.size accordingly, but if the default for
batch.initial.size < batch.size it could cause regressions for existing
users with a large record size, I think. It should be enough for
batch.initial.size to default to batch.size, allowing users who care about
the memory saving in the off-peak throughput case to do the tuning, but not
causing a regression for existing users.

I think this KIP would change the behaviour of producers when there are
multiple partitions ready to be sent: By sending all the ready buffers
(which may now be > batch.size) for the first partition, we could end up
excluding ready buffers for other partitions from the current send. In
other words, as I understand the KIP currently, there's a change in
fairness. I think the code in RecordAccumulator#drainBatchesForOneNode will
ensure fairness in the long run, because the drainIndex will ensure that
those other partitions each get their turn at being the first. But isn't
there the risk that drainBatchesForOneNode would end up not sending ready
batches well past when they ought to be sent (according to their linger.ms),
because it's sending buffers for earlier partitions too aggressively? Or,
to put it another way, perhaps the RecordAccumulator should round-robin the
ready buffers for _all_ the partitions before trying to fill the remaining
space with the extra buffers (beyond the batch.size limit) for the first
partitions?

Kind regards,

Tom

On Wed, Oct 20, 2021 at 1:35 PM Luke Chen  wrote:

> Hi Ismael and all devs,
> Is there any comments/suggestions to this KIP?
> If no, I'm going to update the KIP based on my previous mail, and start a
> vote tomorrow or next week.
>
> Thank you.
> Luke
>
> On Mon, Oct 18, 2021 at 2:40 PM Luke Chen  wrote:
>
> > Hi Ismael,
> > Thanks for your comments.
> >
> > 1. Why do we have to reallocate the buffer? We can keep a list of buffers
> > instead and avoid reallocation.
> > -> Do you mean we allocate multiple buffers with "buffer.initial.size",
> > and link them together (with linked list)?
> > ex:
> > a. We allocate 4KB initial buffer
> > | 4KB |
> >
> > b. when new records reached and the remaining buffer is not enough for
> the
> > records, we create another batch with "batch.initial.size" buffer
> > ex: we already have 3KB of data in the 1st buffer, and here comes the 2KB
> > record
> >
> > | 4KB (1KB remaining) |
> > now, record: 2KB coming
> > We fill the 1st 1KB into 1st buffer, and create new buffer, and linked
> > together, and fill the rest of data into it
> > | 4KB (full) | ---> | 4KB (3KB remaining) |
> >
> > Is that what you mean?
> > If so, I think I like this idea!
> > If not, please explain more detail about it.
> > Thank you.
> >
> > 2. I think we should also consider tweaking the semantics of batch.size
> so
> > that the sent batches can be larger if the batch is not ready to be sent
> > (while still respecting max.request.size and perhaps a new
> max.batch.size).
> >
> > --> In the KIP, I was trying to make the "batch.size" as the upper bound
> > of the batch size, and introduce a "batch.initial.size" as initial batch
> > size.
> > So are you saying that we can let "batch.size" as initial batch size and
> > introduce a "max.batch.size" as upper bound value?
> > That's a good suggestion, but that would change the semantics of
> > "batch.size", which might surprise some users. I think my original
> proposal
> > ("batch.initial.size") is safer for users. What do you think?
> >
> > Thank you.
> > Luke
> >
> >
> > On Mon, Oct 18, 2021 at 3:12 AM Ismael Juma  wrote:
> >
> >> I think we should also consider tweaking the semantics of batch.size so
> >> that the sent batches can be larger if the batch is not ready to be sent
> >> (while still respecting max.request.size and perhaps a new
> >> max.batch.size).
> >>
> >> Ismael
> >>
> >> On Sun, Oct 17, 2021, 12:08 PM Ismael Juma  wrote:
> >>
> >> > Hi Luke,
> >> >
> >> > Thanks for the KIP. Why do we have to reallocate the buffer? We can
> >> keep a
> >> > list of buffers instead and avoid reallocation.
> >> >
> >> > Ismael
> >> >
> >> > On Sun, Oct 17, 2021, 2:02 AM Luke Chen  wrote:
> >> >
> >> >> Hi Kafka dev,
> >> >> I'd like to start the discussion for the proposal: KIP-782:
> Expandable
> >> >> batch size in producer.
> >> >>
> >> >> The main purpose for this KIP is to have better memory usage in
> >> producer,
> >> >> and also save users from the dilemma while setting the batch size
> >> >> configuration. After this KIP, users can set a higher batch.size
> >> without
> >> >> worries, and of course, with an appropriate 

[VOTE] KIP-797: Accept duplicate listener on port for IPv4/IPv6

2021-11-22 Thread Matthew de Detrich
Hello everyone,

I would like to start a vote for KIP-797: Accept duplicate listener on port
for IPv4/IPv6

https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=195726330

The purpose of the KIP is to loosen current validation for non advertised
listeners so that you can have an IPv4 address and an IPv6 address on the
same port. All other behaviour remains the same as before (since these are
disparate IP stacks there are no technical reasons not to allow this).

PR is located at https://github.com/apache/kafka/pull/11478

Comments and feedback are welcome!

Regards

-- 

Matthew de Detrich

*Aiven Deutschland GmbH*

Immanuelkirchstraße 26, 10405 Berlin

Amtsgericht Charlottenburg, HRB 209739 B

Geschäftsführer: Oskari Saarenmaa & Hannu Valtonen

*m:* +491603708037

*w:* aiven.io *e:* matthew.dedetr...@aiven.io


Re: [DISCUSS] KIP-714: Client metrics and observability

2021-11-22 Thread Viktor Somogyi-Vass
Hi Magnus,

I think this is a very useful addition. We also have a similar (but much
more simplistic) implementation of this. Maybe I missed it in the KIP but
what about adding metrics about the subscription cache itself? That I think
would improve its usability and debuggability as we'd be able to see its
performance, hit/miss rates, eviction counts and others.

Best,
Viktor

On Thu, Nov 18, 2021 at 5:12 PM Magnus Edenhill  wrote:

> Hi Mickael,
>
> see inline.
>
> Den ons 10 nov. 2021 kl 15:21 skrev Mickael Maison <
> mickael.mai...@gmail.com
> >:
>
> > Hi Magnus,
> >
> > I see you've addressed some of the points I raised above but some (4,
> > 5) have not been addressed yet.
> >
>
> Re 4) How will the user/app know metrics are being sent.
>
> One possibility is to add a JMX metric (thus for user consumption) for the
> number of metric pushes the
> client has performed, or perhaps the number of metrics subscriptions
> currently being collected.
> Would that be sufficient?
>
> Re 5) Metric sizes and rates
>
> A worst case scenario for a producer that is producing to 50 unique topics
> and emitting all standard metrics yields
> a serialized size of around 100KB prior to compression, which compresses
> down to about 20-30% of that depending
> on compression type and topic name uniqueness.
> The numbers for a consumer would be similar.
>
> In practice the number of unique topics would be far less, and the
> subscription set would typically be for a subset of metrics.
> So we're probably closer to 1kb, or less, compressed size per client per
> push interval.
>
> As both the subscription set and push intervals are controlled by the
> cluster operator it shouldn't be too hard
> to strike a good balance between metrics overhead and granularity.
>
>
>
> >
> > I'm really uneasy with this being enabled by default on the client
> > side. When collecting data, I think the best practice is to ensure
> > users are explicitly enabling it.
> >
>
> Requiring metrics to be explicitly enabled on clients severely cripples its
> usability and value.
>
> One of the problems that this KIP aims to solve is for useful metrics to be
> available on demand
> regardless of the technical expertise of the user. As Ryanne points, out a
> savvy user/organization
> will typically have metrics collection and monitoring in place already, and
> the benefits of this KIP
> are then more of a common set and format metrics across client
> implementations and languages.
> But that is not the typical Kafka user in my experience, they're not Kafka
> experts and they don't have the
> knowledge of how to best instrument their clients.
> Having metrics enabled by default for this user base allows the Kafka
> operators to proactively and reactively
> monitor and troubleshoot client issues, without the need for the less savvy
> user to do anything.
> It is often too late to tell a user to enable metrics when the problem has
> already occurred.
>
> Now, to be clear, even though metrics are enabled by default on clients it
> is not enabled by default
> on the brokers; the Kafka operator needs to build and set up a metrics
> plugin and add metrics subscriptions
> before anything is sent from the client.
> It is opt-out on the clients and opt-in on the broker.
>
>
>
>
> > You mentioned brokers already have
> > some(most?) of the information contained in metrics, if so then why
> > are we collecting it again? Surely there must be some new information
> > in the client metrics.
> >
>
> From the user's perspective the Kafka infrastructure extends from
> producer.send() to
> messages being returned from consumer.poll(), a giant black box where
> there's a lot going on between those
> two points. The brokers currently only see what happens once those requests
> and messages hits the broker,
> but as Kafka clients are complex pieces of machinery there's a myriad of
> queues, timers, and state
> that's critical to the operation and infrastructure that's not currently
> visible to the operator.
> Relying on the user to accurately and timely provide this missing
> information is not generally feasible.
>
>
> Most of the standard metrics listed in the KIP are data points that the
> broker does not have.
> Only a small number of metrics are duplicates (like the request counts and
> sizes), but they are included
> to ease correlation when inspecting these client metrics.
>
>
>
> > Moreover this is a brand new feature so it's even harder to justify
> > enabling it and forcing onto all our users. If disabled by default,
> > it's relatively easy to enable in a new release if we decide to, but
> > once enabled by default it's much harder to disable. Also this feature
> > will apply to all future metrics we will add.
> >
>
> I think maturity of a feature implementation should be the deciding factor,
> rather than
> the design of it (which this KIP is). I.e., if the implementation is not
> deemed mature enough
> for release X.Y it will be disabled.
>
>
>
> > Overall I think 

Re: [DISCUSS] KIP-784: Add top-level error code field to DescribeLogDirsResponse

2021-11-22 Thread Magnus Edenhill
Hey Mickael and thanks for the KIP.

I'm a bit late to the game, but one small nit for consistency:
The global response error codes are typically at the start of the response,
usually right after the ThrottleTime.
This also makes it easier for the response parser to assign the global
error for each sub-result, if needed.

Thanks,
Magnus


Den mån 18 okt. 2021 kl 17:08 skrev Mickael Maison :

> Hi,
>
> I wrote a very minor KIP that proposes adding a top-level error code
> field to DescribeLogDirsResponse:
>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-784%3A+Add+top-level+error+code+field+to+DescribeLogDirsResponse
>
> Please let me know if you have any feedback.
> Thanks
>


Re: [VOTE] KIP-798 Add possibility to write kafka headers in Kafka Console Producer

2021-11-22 Thread Tom Bentley
Hi Florin,

Thanks for the KIP!

+1 (binding),

Kind regards,

Tom

On Mon, Nov 22, 2021 at 6:51 AM Luke Chen  wrote:

> Hi Florin,
> Thanks for the KIP.
>
> This KIP makes sense to me. Just a comment that the motivation section is
> not clearly explain why this KIP is important.
> I think John already mentioned a good motivation, which is to support "not
> only UTF-8".
> You should put that into the KIP, and of course if you have other thoughts,
> please also add them into KIP.
>
> Also, in the "public interface" section, there are 3 "Default parsing
> pattern", I think you should add 1 remaining case (false, false) to make it
> complete.
>
> Otherwise, look good to me.
>
> Thank you.
> Luke
>
>
> On Sun, Nov 21, 2021 at 7:37 PM Florin Akermann  >
> wrote:
>
> > Hi John,
> >
> > Thanks for the vote and feedback.
> >
> > The thought occurred to me too.
> >
> > Do I understand it correctly: the current version of the
> > kafka-console-producer cannot be used for anything other than UTF-8 keys
> > and values?
> > (There is no other implementation of MessageReader other than the
> > ConsoleProducer$LineMessageReader)
> > In other words, currently users seem to only apply it with utf-8 strings
> > for keys and values?
> > This is why I figured I would not deviate from this assumption solely for
> > the headers.
> >
> > I will happily raise another KIP / Jira if there is a need to specify
> other
> > formats / serializers for headers, keys and/or values.
> >
> > Thanks,
> > Florin
> >
> >
> > On Sat, 20 Nov 2021 at 19:34, John Roesler  wrote:
> >
> > > Hi Florin,
> > >
> > > Thanks for the KIP!
> > >
> > > I think the assumption that header values are UTF-8 strings might not
> > hold
> > > up in the long run, but it seems like we can easily add a property
> later
> > to
> > > specify the format. It seems like this scope is probably a handy
> addition
> > > on its own.
> > >
> > > I’m +1 (binding)
> > >
> > > Thanks,
> > > John
> > >
> > >
> > > On Fri, Nov 19, 2021, at 15:06, flo wrote:
> > > > <
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-798%3A+Add+possibility+to+write+kafka+headers+in+Kafka+Console+Producer
> > > >
> > >
> >
>


Re: [VOTE] KIP-784: Add top-level error code field to DescribeLogDirsResponse

2021-11-22 Thread Tom Bentley
Hi Mickael,

It's pretty low value, but I think consistency is a useful trait, and it's
easily achievable here.

+1 (binding).

Kind regards,

Tom


On Thu, Nov 18, 2021 at 2:56 PM Mickael Maison 
wrote:

> Bumping this thread.
>
> Let me know if you have any feedback.
>
> Thanks,
> Mickael
>
> On Wed, Oct 27, 2021 at 3:25 PM Luke Chen  wrote:
> >
> > Hi Mickael,
> > Thanks for the KIP.
> > It's good to keep it consistent with others, to have top-level error
> field.
> >
> > + 1 (non-binding)
> >
> > Thank you.
> > Luke
> >
> > On Wed, Oct 27, 2021 at 9:01 PM Mickael Maison  >
> > wrote:
> >
> > > Hi all,
> > >
> > > I'd like to start the vote on this minor KIP.
> > >
> > >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-784%3A+Add+top-level+error+code+field+to+DescribeLogDirsResponse
> > >
> > > Please take a look, vote or let me know if you have any feedback.
> > >
> > > Thanks
> > >
>
>


Re: [DISCUSS] KIP-792: Add "generation" field into consumer protocol

2021-11-22 Thread Luke Chen
Hello David,

For (3):



* I suppose that we could add a `generation` field to the JoinGroupRequest
instead to do the fencing that you describe while handling the sentinel in
the assignor directly. If we would add the `generation` to the request
itself, would we need the `generation` in the subscription protocol as
well?*

On second thought, I think this is not better than adding `generation`
field in the subscription protocol, because I think we don't have to do any
generation validation on joinGroup request. The purpose of
`joinGroupRequest` is to accept any members to join this group, even if the
member is new or ever joined or what. As long as we have the generationId
in the subscription metadata, the consumer lead can leverage the info to
ignore the old ownedPartitions (or do other handling), and the rebalance
can still complete successfully and correctly. On the other hand, if we did
the generation check on JoinGroupRequest, and return `ILLEGAL_GENERATION`
back to consumer, the consumer needs to clear its generation info and
rejoin the group to continue the rebalance. It needs more request/response
network and slow down the rebalance.

So I think we should add the `generationId` field into Subscription
protocol to achieve what we want.

Thank you.
Luke

On Thu, Nov 18, 2021 at 8:51 PM Luke Chen  wrote:

> Hi David,
> Thanks for your feedback.
>
> I've updated the KIP for your comments (1)(2).
> For (3), it's a good point! Yes, we didn't deserialize the subscription
> metadata on broker side, and it's not necessary to add overhead on broker
> side. And, yes, I think we can fix the original issue by adding a
> "generation" field into `JoinGroupRequest` instead, and also add a field
> into `JoinGroupResponse` in `JoinGroupResponseMember` field. That way, the
> broker can identify the old member from `JoinGroupRequest`. And the
> assignor can also get the "generation" info via the `Subscription` instance.
>
> I'll update the KIP to add "generation" field into `JoinGroupRequest` and
> `JoinGroupResponse`, if there is no other options.
>
> Thank you.
> Luke
>
>
> On Tue, Nov 16, 2021 at 12:31 AM David Jacot 
> wrote:
>
>> Hi Luke,
>>
>> Thanks for the KIP. Overall, I think that the motivation makes sense. I
>> have a couple of comments/questions:
>>
>> 1. In the Public Interfaces section, it would be great if you could put
>> the
>> end state not the current one.
>>
>> 2. Do we need to update the Subscription class to expose the
>> generation? If so, it would be great to mention it in the Public
>> Interfaces section as well.
>>
>> 3. You mention that the broker will set the generation if the subscription
>> contains a sentinel value (-1). As of today, the broker does not parse
>> the subscription so I am not sure how/why we would do this. I suppose
>> that we could add a `generation` field to the JoinGroupRequest instead
>> to do the fencing that you describe while handling the sentinel in the
>> assignor directly. If we would add the `generation` to the request itself,
>> would we need the `generation` in the subscription protocol as well?
>>
>> Best,
>> David
>>
>> On Fri, Nov 12, 2021 at 3:31 AM Luke Chen  wrote:
>> >
>> > Hi all,
>> >
>> > I'd like to start the discussion for KIP-792: Add "generation" field
>> into
>> > consumer protocol.
>> >
>> > The goal of this KIP is to allow assignor/consumer coordinator/group
>> > coordinator to have a way to identify the out-of-date
>> members/assignments.
>> >
>> > Detailed description can be found here:
>> >
>> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=191336614
>> >
>> > Any feedback is welcome.
>> >
>> > Thank you.
>> > Luke
>>
>