Hi Tom,
Thanks for the KIP. I am aware that the voting thread was started, but
wanted to discuss couple of concerns here first.
I think the coupling of RequestedTopicState#generatedReplicaAssignment()
and TopicState#replicasAssignments() does not work well in case where the
request deals only w
Hi Tom,
Just wanted to check what you think about the comments I made in my last
message. I think this KIP is a big improvement to our current policy
interfaces, and really hope we can get this KIP in.
Thanks,
Anna
On Thu, May 31, 2018 at 3:29 PM, Anna Povzner wrote:
> Hi Tom,
>
>
Hi Jason and Dong,
I’ve been thinking about your suggestions and discussion regarding
position(), seek(), and new proposed API.
Here is my thought process why we should keep position() and seek() API
unchanged.
I think we should separate {offset, leader epoch} that uniquely identifies
a messa
ot; use-case, it seems that there can be two
> positions, i.e. earliest and latest. So these two cases can be
> Consumer.fulfilled by seekToBeginning() and Consumer.seekToEnd(). Then it
> seems that user will only need to call position() and seek() for "seeking
> to a message"
...), then user can probably just call
> Consumer#seekToEnd() without calling position() and seek(...). Similarly
> user can call Consumer#seekToBeginning() to the seek to the earliest
> position without calling position() and seek(...). Thus position() only
> needs to return the actual consumed
n that the consumer will not
be able to detect log truncation in all cases?
Thanks,
Anna
On Tue, Jul 10, 2018 at 2:20 PM Anna Povzner wrote:
> Hi Dong,
>
> Thanks for the follow up! I finally have much more clear understanding of
> where you are coming from.
>
> You are right
hand, if we use seek(offset), consumer will
> not detect log truncation in some cases which weakens the guarantee of this
> KIP. Does this make sense?
>
> Thanks,
> Dong
>
> On Tue, Jul 10, 2018 at 3:14 PM, Anna Povzner wrote:
>
> > Sorry, I hit "send" befo
> > problems, but I am still verifying the model.
> >
> > Thanks,
> > Jason
> >
> >
> > On Wed, Jul 11, 2018 at 10:45 AM, Dong Lin wrote:
> >
> > > Hey Anna,
> > >
> > > Thanks much for the explanation. Approach 1 also sound
%3A+Allow+fetchers+to+detect+and+handle+log+truncation)
> > with a section say "workflow" on how the proposed API will be co-used
> with
> > others to:
> >
> > 1. consumer callers handling a LogTruncationException.
> > 2. consumer internals for handling a
Hi,
I just created KIP-279 to fix edge cases of log divergence for both clean
and unclean leader election configs.
https://cwiki.apache.org/confluence/display/KAFKA/KIP-279%3A+Fix+log+divergence+between+leader+and+follower+after+fast+leader+fail+over
The KIP is basically a follow up to KIP-10
> Though the size increases, the number of roundtrips is reduced meaningfully
> which would increase the robustness of the solution.
>
> Please expand the reasoning for unclean leader election for this
> alternative.
>
> Thanks
>
> On Thu, Apr 5, 2018 at 12:17 PM, Anna
for convergence, far
> and away the common case would only require a very small number (usually
> exactly one).
>
> Is that correct?
>
> Thanks,
> Jason
>
> On Fri, Apr 6, 2018 at 5:47 PM, Ted Yu wrote:
>
> > Makes sense.
> > Thanks for the explanation.
&
in a compacted topic by high
> watermark. Filed https://issues.apache.org/jira/browse/KAFKA-6780 to track
> it.
>
> Jun
>
>
> On Thu, Apr 5, 2018 at 12:17 PM, Anna Povzner wrote:
>
> > Hi,
> >
> >
> > I just created KIP-279 to fix edge cases of log di
Hi All,
I would like to start the vote on KIP-279: Fix log divergence between
leader and follower after fast leader fail over.
For reference, here's the KIP wiki:
https://cwiki.apache.org/confluence/display/KAFKA/KIP-279%3A+Fix+log+divergence+between+leader+and+follower+after+fast+leader+fail+o
g a leader change. That's the
> only impact, right?
>
> best,
> Colin
>
> On Mon, Apr 16, 2018, at 09:48, Ismael Juma wrote:
> > Thanks for the detailed KIP. +1 (binding)
> >
> > Ismael
> >
> > On Sat, Apr 14, 2018 at 3:54 PM, Anna Povzner wrote:
>
Guozhang, thanks for catching this, I fixed the description (the example
assumed response with 21, '11' was a typo).
On Tue, Apr 17, 2018 at 1:25 PM, Anna Povzner wrote:
> Hi Colin,
>
> Yes, the impact of "losing" entries in the LeaderEpoch file is more
> roun
The vote for KIP-279 has passed with 5 binding and 2 non-binding +1s (and
no objections).
Thanks everyone for your reviews and feedback,
Anna
On Tue, Apr 17, 2018 at 1:49 PM, Anna Povzner wrote:
> Guozhang, thanks for catching this, I fixed the description (the example
> assumed re
+1
Thanks for the KIP!
On Thu, Aug 9, 2018 at 5:16 PM Jun Rao wrote:
> Hi, Jason,
>
> Thanks for the KIP. +1 from me.
>
> Jun
>
> On Wed, Aug 8, 2018 at 1:04 PM, Jason Gustafson
> wrote:
>
> > Hi All,
> >
> > I'd like to start a vote for KIP-320:
> > https://cwiki.apache.org/confluence/display
Hi,
I am opening the voting thread for KIP-32: Add CreateTime and
LogAppendTime to Kafka message.
For reference, here's the KIP wiki:
https://cwiki.apache.org/confluence/display/KAFKA/KIP-32+-+Add+CreateTime+and+LogAppendTime+to+Kafka+message
And the mailing list threads:
September:
http://mai
ils on all the alternatives
> and
> > how we arrived at the proposal. Its really great!
> >
> > Can you point me at where the migration plan was discussed? It looks
> overly
> > complex and I have a bunch of questions, but if there was a discussion,
> I'd
> > like
ot; clients are upgraded, bump format.version to 1
> > > >>> (rolling).
> > > >>>>>>
> > > >>>>>> Becket, can you confirm?
> > > >>>>>>
> > > >>>>>> Assuming this is the process, I'm +1 on t
Hi,
I just created a KIP-42 for adding producer and consumer interceptors for
intercepting messages at different points on producer and consumer.
https://cwiki.apache.org/confluence/display/KAFKA/KIP-42%3A+Add+Producer+and+Consumer+Interceptors
Comments and suggestions are welcome!
Thanks,
Anna
t; > > > > > James,
> > > > > >
> > > > > > That is one of the many monitoring use cases for the interceptor
> > > > > interface.
> > > > > >
> > > > > > Thanks,
> > > > > > Neha
> > >
t is there a plan to expose an API to
> check if we have messages in RecordAccumulator for a particular topic? I
> have a usecase that I can discuss if we plan for this API.
>
> Thanks,
>
> Mayuresh
>
>
> On Mon, Jan 25, 2016 at 4:49 PM, Anna Povzner wrote:
>
> >
gt; > > > > > James,
> > > > > >
> > > > > > That is one of the many monitoring use cases for the interceptor
> > > > > interface.
> > > > > >
> > > > > > Thanks,
> > > > &
apping between
> > > > > > > ProducerInterceptor.onAcknowledgement() and the producer
> > callback?
> > > I
> > > > > can
> > > > > > > see that the Callback could be a per record setting while
> > > > > > > onAcknowledgemen
a of the interceptor is that it
> > > > doesn't
> > > > > > > > require any code change in apps so you can globally add
> > behavior
> > > to
> > > > > > your
> > > > > > > > Kafka usage without changing app code.
I will send more detailed meeting notes regarding KIP-42 in the KIP-42
discussion thread.
Thanks,
Anna
On Tue, Jan 26, 2016 at 1:21 PM, Jun Rao wrote:
> The following are the notes from today's KIP discussion.
>
>
>- KIP-42: We agreed to leave the broker side interceptor for another
>KI
or to
> be produced to cluster B, we can safely delete the topic in cluster B
> without disturbing the pipeline.
>
> Thanks,
>
> Mayuresh
>
> On Tue, Jan 26, 2016 at 10:31 AM, Anna Povzner wrote:
>
> > Thanks Ismael and Todd for your feedback!
> >
> > I a
> allows for the use cases we can envision right now, and I think
> that’s
> > > > going to provide a great deal of opportunity for implementing things
> > like
> > > > audit, especially within a multi-tenant environment. Looking forward
> to
> > >
to the above.
Thanks,
Anna
On Wed, Jan 27, 2016 at 2:56 PM, Anna Povzner wrote:
> Hi Mayuresh,
>
> I see why you would want to check for messages left in the
> RecordAccumulator. However, I don't think this will completely solve the
> problem. Messages could be in-flight
s for being diligent.
> >
> > +1 on #1.2 and sounds good on #3. I recommend adding checksum and size
> > fields to RecordMetadata and ConsumerRecord instead of exposing metadata
> > piecemeal in the interceptor APIs.
> >
> > Thanks,
> > Neha
> >
> > O
r serialization, i.e. application bytes. (The
> uncompressed record size can be easily derived as well)
> 2. The actual bytes sent over the wire.
> We can get (1) easily, but (2) is difficult to get at Record level when we
> use compression.
>
> Thanks,
>
> Jiangjie (Becket) Qin
>
l that
> can provide this? The other option is to estimate it (such as making an
> assumption that the messages in a batch are equal in size, which is not
> necessarily true), which is probably not the right answer.
>
> -Todd
>
>
> On Thu, Jan 28, 2016 at 1:48 PM, Anna Povzner
gs to discuss.
Thanks,
Anna
On Thu, Jan 28, 2016 at 2:25 PM, Anna Povzner wrote:
> Regarding record size as bytes sent over the wire, my concern is that it
> is almost impossible to calculate per-record. We could do as: 1) compressed
> bytes / number of records in a compressed me
tition + Offset already uniquely identified a
> > > message?
> > > > It seems better than CRC no matter from summary point of view or
> > auditing
> > > > point of view.
> > > >
> > > > 2. Currently CRC only has 4 byte
Joel, thanks for your feedback. I updated the wiki based on your comments
about the wiki writeup.
On Fri, Jan 29, 2016 at 11:50 AM, Anna Povzner wrote:
> Becket,
>
> In your scenario with one message from producer A and one message from
> producer B, those are two different messag
Hi All,
I am opening the voting thread for KIP-42: Add Producer and Consumer
Interceptors.
For reference, here's the KIP wiki:
https://cwiki.apache.org/confluence/display/KAFKA/KIP-42%3A+Add+Producer+and+Consumer+Interceptors
And the mailing list threads:
January:
http://mail-archives.apache.org
Hi All,
I am opening the voting thread for KIP-42: Add Producer and Consumer
Interceptors.
For reference, here's the KIP wiki:
https://cwiki.apache.org/confluence/display/KAFKA/KIP-42%3A+Add+Producer+and+Consumer+Interceptors
And the mailing list threads:
January:
http://mail-archives.apache.org
hy wrote:
>
> > +1
> >
> > On Monday, February 1, 2016, Anna Povzner wrote:
> >
> > > Hi All,
> > >
> > > I am opening the voting thread for KIP-42: Add Producer and Consumer
> > > Interceptors.
> > >
> > > For reference
have any concerns.
Thanks,
Anna
On Thu, May 21, 2020 at 5:42 PM Anna Povzner wrote:
> The vote for KIP-612 has passed with 3 binding and 3 non-binding +1s, and
> no objections.
>
>
> Thanks everyone for reviews and feedback,
>
> Anna
>
> On Tue, May 19, 2020 at 2:41
Hi David and Jun,
I wanted to add to the discussion about using requests/sec vs. time on
server threads (similar to request quota) for expressing quota for topic
ops.
I think request quota does not protect the brokers from overload by itself
-- it still requires tuning and sometimes re-tuning, be
Hi,
I just created KIP-612 to allow limiting connection creation rate on
brokers, and would like to start a discussion.
https://cwiki.apache.org/confluence/display/KAFKA/KIP-612%3A+Ability+to+Limit+Connection+Creation+Rate+on+Brokers
Feedback and suggestions are welcome!
Thanks,
Anna
the number of connections per IP? Instead of rate limiting all the
> > creation,
> > we could perhaps rate limit the number of creation per IP as well. That
> > could
> > perhaps reduce the effect on the other clients. That may be harder to
> > implement
> >
them until the rate is back within the rate limit.
On Wed, May 13, 2020 at 11:46 AM Anna Povzner wrote:
> Hi David and Alexandre,
>
> Thanks so much for your feedback! Here are my answers:
>
> 1. Yes, we have seen several cases of clients that create a new connection
> per produ
> dropping doesn't really help since that IP is going to reconnect. If we
> > want to rate limit what happens next, i.e. authentication, then
> > throttling the accepted connection so its processing is delayed would
> > perhaps be better?
> >
> > Regards,
> &g
/tree/trunk/tools/src/main/java/org/apache/kafka/trogdor/workload
Regards,
Anna
On Mon, May 18, 2020 at 8:57 AM Rajini Sivaram
wrote:
> Hi Anna,
>
> Thanks for the response, sounds good.
>
> Regards,
>
> Rajini
>
>
> On Sun, May 17, 2020 at 1:38 AM Anna P
Just want to clarify what's the expected behavior to be seen on the
> client side?
>
> On Mon, May 18, 2020 at 12:04 PM Anna Povzner wrote:
>
> > Hi Alexandre,
> >
> > Thanks for your comments. My answers are below:
> >
> > 900. The KIP does not propose
Thanks everyone for the feedback. I will start a voting thread tomorrow
morning if there are no more comments.
Regards,
Anna
On Mon, May 18, 2020 at 2:06 PM Anna Povzner wrote:
> Hi Boyang,
>
> This KIP does not change the protocol with clients. The behavior is the
> same as with K
I realized the KIP freeze is on May 20. I will start the voting thread now.
On Mon, May 18, 2020 at 3:19 PM Anna Povzner wrote:
> Thanks everyone for the feedback. I will start a voting thread tomorrow
> morning if there are no more comments.
>
> Regards,
> Anna
>
> On Mon,
Hi All,
I would like to start the vote on KIP-612: Ability to limit connection
creation rate on brokers.
For reference, here is the KIP wiki:
https://cwiki.apache.org/confluence/display/KAFKA/KIP-612%3A+Ability+to+Limit+Connection+Creation+Rate+on+Brokers
And discussion thread:
https://lists.apa
the nice feature to control the connection creation
> > rate
> > > > from the clients.
> > > >
> > > > On Tue, May 19, 2020 at 8:16 AM Gwen Shapira
> > wrote:
> > > >
> > > > > +1 (binding)
> > > > >
> &
Hi Jun and David,
Regarding token bucket vs, Rate behavior. We recently observed a couple of
cases where a bursty workload behavior would result in long-ish pauses in
between, resulting in lower overall bandwidth than the quota. I will need
to debug this a bit more to be 100% sure, but it does loo
> > > >
> > > > > Hi Jun and Anna,
> > > > >
> > > > > Thank you both for your replies.
> > > > >
> > > > > Based on our recent discussion, I agree that using a rate is better
> > to
> > > > > remain
few insights.
> > I don't have specific numbers that can be used out of the box so I
> > think that it is better to not put any for the time being. We can always
> > complement later on in the documentation.
> >
> > Please, take a look and let me know what you thi
+1 (not binding)
Thanks for the KIP!
-Anna
On Thu, Jun 4, 2020 at 8:26 AM Mickael Maison
wrote:
> +1 (binding)
> Thanks David for looking into this important issue
>
> On Thu, Jun 4, 2020 at 3:59 PM Tom Bentley wrote:
> >
> > +1 (non binding).
> >
> > Thanks!
> >
> > On Wed, Jun 3, 2020 at 3:
Hi Brian,
The KIP looks good!
I have one clarification question regarding the distinction between
describe and resolve API. Suppose I set request quota for
/config/users/”user-1”/clients/"client-1" to 100 and request quota for
/config/users/”user-1” to 200. Is this correct that describeClientQuot
Hi Bill,
Regarding KIP-612, only the first half of the KIP will get into 2.7
release: Broker-wide and per-listener connection rate limits, including
corresponding configs and metric (KAFKA-10023). I see that the table in the
release plan tags KAFKA-10023 as "old", not sure what it refers to. Note
in the bucket, in addition to the current rate
> metric. It would be great to add it in the metrics section of the KIP as
> well
> for completeness.
>
> Best,
> David
>
> On Tue, Aug 11, 2020 at 4:28 AM Anna Povzner wrote:
>
> > Hi All,
> >
> >
Anna Povzner created KAFKA-7786:
---
Summary: Fast update of leader epoch may stall partition fetching
due to FENCED_LEADER_EPOCH
Key: KAFKA-7786
URL: https://issues.apache.org/jira/browse/KAFKA-7786
[
https://issues.apache.org/jira/browse/KAFKA-6795?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Anna Povzner resolved KAFKA-6795.
-
Resolution: Fixed
Fix Version/s: 2.0.0
> Add unit test for ReplicaAlterLogDirsThr
Anna Povzner created KAFKA-6975:
---
Summary: AdminClient.deleteRecords() may cause replicas unable to
fetch from beginning
Key: KAFKA-6975
URL: https://issues.apache.org/jira/browse/KAFKA-6975
Project
Anna Povzner created KAFKA-7104:
---
Summary: ReplicaFetcher thread may die because of inconsistent log
start offset in fetch response
Key: KAFKA-7104
URL: https://issues.apache.org/jira/browse/KAFKA-7104
Anna Povzner created KAFKA-7150:
---
Summary: Error in processing fetched data for one partition may
stop follower fetching other partitions
Key: KAFKA-7150
URL: https://issues.apache.org/jira/browse/KAFKA-7150
Anna Povzner created KAFKA-7151:
---
Summary: Broker running out of disk space may result in state
where unclean leader election is required
Key: KAFKA-7151
URL: https://issues.apache.org/jira/browse/KAFKA-7151
Anna Povzner created KAFKA-6693:
---
Summary: Add Consumer-only benchmark workload to Trogdor
Key: KAFKA-6693
URL: https://issues.apache.org/jira/browse/KAFKA-6693
Project: Kafka
Issue Type
[
https://issues.apache.org/jira/browse/KAFKA-6693?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Anna Povzner resolved KAFKA-6693.
-
Resolution: Fixed
https://github.com/apache/kafka/pull/4775
> Add Consumer-only benchm
Anna Povzner created KAFKA-6795:
---
Summary: Add unit test for ReplicaAlterLogDirsThread
Key: KAFKA-6795
URL: https://issues.apache.org/jira/browse/KAFKA-6795
Project: Kafka
Issue Type
Anna Povzner created KAFKA-6809:
---
Summary: connections-created metric does not behave as expected
Key: KAFKA-6809
URL: https://issues.apache.org/jira/browse/KAFKA-6809
Project: Kafka
Issue
Anna Povzner created KAFKA-6823:
---
Summary: Transient failure in
DynamicBrokerReconfigurationTest.testThreadPoolResize
Key: KAFKA-6823
URL: https://issues.apache.org/jira/browse/KAFKA-6823
Project
Anna Povzner created KAFKA-6824:
---
Summary: Transient failure in
DynamicBrokerReconfigurationTest.testAddRemoveSslListener
Key: KAFKA-6824
URL: https://issues.apache.org/jira/browse/KAFKA-6824
Project
Anna Povzner created KAFKA-6859:
---
Summary: Follower should not send OffsetForLeaderEpoch for
undefined leader epochs
Key: KAFKA-6859
URL: https://issues.apache.org/jira/browse/KAFKA-6859
Project: Kafka
Anna Povzner created KAFKA-7415:
---
Summary: OffsetsForLeaderEpoch may incorrectly respond with
undefined epoch causing truncation to HW
Key: KAFKA-7415
URL: https://issues.apache.org/jira/browse/KAFKA-7415
Anna Povzner created KAFKA-7923:
---
Summary: Add unit test to verify Kafka-7401 in AK versions >= 2.0
Key: KAFKA-7923
URL: https://issues.apache.org/jira/browse/KAFKA-7923
Project: Kafka
Is
Anna Povzner created KAFKA-8001:
---
Summary: Fetch from future replica stalls when local replica
becomes a leader
Key: KAFKA-8001
URL: https://issues.apache.org/jira/browse/KAFKA-8001
Project: Kafka
Anna Povzner created KAFKA-8002:
---
Summary: Replica reassignment to new log dir may not complete if
future and current replicas segment files have different base offsets
Key: KAFKA-8002
URL: https
Anna Povzner created KAFKA-8480:
---
Summary: Clients may fetch incomplete set of topic partitions
during cluster startup
Key: KAFKA-8480
URL: https://issues.apache.org/jira/browse/KAFKA-8480
Project
Anna Povzner created KAFKA-8481:
---
Summary: Clients may fetch incomplete set of topic partitions just
after topic is created
Key: KAFKA-8481
URL: https://issues.apache.org/jira/browse/KAFKA-8481
Project
Anna Povzner created KAFKA-8526:
---
Summary: Broker may select a failed dir for new replica even in
the presence of other live dirs
Key: KAFKA-8526
URL: https://issues.apache.org/jira/browse/KAFKA-8526
Anna Povzner created KAFKA-8782:
---
Summary: ReplicationQuotaManagerTest and ClientQuotaManagerTest
should close Metrics object
Key: KAFKA-8782
URL: https://issues.apache.org/jira/browse/KAFKA-8782
[
https://issues.apache.org/jira/browse/KAFKA-3777?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Work on KAFKA-3777 started by Anna Povzner.
---
> Extract the existing LRU cache out of RocksDBSt
[
https://issues.apache.org/jira/browse/KAFKA-4691?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15838829#comment-15838829
]
Anna Povzner commented on KAFKA-4691:
-
I agree with [~mjsax] about not chan
[
https://issues.apache.org/jira/browse/KAFKA-3202?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15189917#comment-15189917
]
Anna Povzner commented on KAFKA-3202:
-
[~enothereska] I think you meant to post
[
https://issues.apache.org/jira/browse/KAFKA-3320?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15213677#comment-15213677
]
Anna Povzner commented on KAFKA-3320:
-
If you look at verifiable_producer.py
Anna Povzner created KAFKA-3555:
---
Summary: Unexpected close of KStreams transformer
Key: KAFKA-3555
URL: https://issues.apache.org/jira/browse/KAFKA-3555
Project: Kafka
Issue Type: Bug
Anna Povzner created KAFKA-3566:
---
Summary: Enable VerifiableProducer and ConsoleConsumer to run with
interceptors
Key: KAFKA-3566
URL: https://issues.apache.org/jira/browse/KAFKA-3566
Project: Kafka
Anna Povzner created KAFKA-3597:
---
Summary: Enable query ConsoleConsumer and VerifiableProducer if
they shutdown cleanly
Key: KAFKA-3597
URL: https://issues.apache.org/jira/browse/KAFKA-3597
Project
[
https://issues.apache.org/jira/browse/KAFKA-3597?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Work on KAFKA-3597 started by Anna Povzner.
---
> Enable query ConsoleConsumer and VerifiableProducer if they shutdown clea
[
https://issues.apache.org/jira/browse/KAFKA-3566?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Work on KAFKA-3566 started by Anna Povzner.
---
> Enable VerifiableProducer and ConsoleConsumer to run with intercept
[
https://issues.apache.org/jira/browse/KAFKA-3597?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Anna Povzner updated KAFKA-3597:
Status: Patch Available (was: In Progress)
> Enable query ConsoleConsumer and VerifiableProdu
[
https://issues.apache.org/jira/browse/KAFKA-2896?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Anna Povzner updated KAFKA-2896:
Status: Patch Available (was: In Progress)
> System test for partition re-assignm
Anna Povzner created KAFKA-3025:
---
Summary: KIP-31 (part 1): Add timestamp field to message, configs,
and Producer/ConsumerRecord
Key: KAFKA-3025
URL: https://issues.apache.org/jira/browse/KAFKA-3025
Anna Povzner created KAFKA-3026:
---
Summary: KIP-32 (part 2): Changes in broker to over-write
timestamp or reject message
Key: KAFKA-3026
URL: https://issues.apache.org/jira/browse/KAFKA-3026
Project
[
https://issues.apache.org/jira/browse/KAFKA-3025?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Anna Povzner updated KAFKA-3025:
Summary: KIP-32 (part 1): Add timestamp field to message, configs, and
Producer/ConsumerRecord
[
https://issues.apache.org/jira/browse/KAFKA-3025?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15068580#comment-15068580
]
Anna Povzner commented on KAFKA-3025:
-
[~lindong] : I am in contact with [~becket
[
https://issues.apache.org/jira/browse/KAFKA-3025?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Anna Povzner updated KAFKA-3025:
Description:
The discussion about this JIRA assignment is still under discussion with
[~becket_qin
[
https://issues.apache.org/jira/browse/KAFKA-3025?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Anna Povzner updated KAFKA-3025:
Description:
The decision about this JIRA assignment is still under discussion with
[~becket_qin
[
https://issues.apache.org/jira/browse/KAFKA-3026?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Anna Povzner updated KAFKA-3026:
Description:
The discussion about this JIRA assignment is still under discussion with
[~becket_qin
[
https://issues.apache.org/jira/browse/KAFKA-3026?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Anna Povzner updated KAFKA-3026:
Description:
The decision about this JIRA assignment is still under discussion with
[~becket_qin
Anna Povzner created KAFKA-3162:
---
Summary: KIP-42: Add Producer and Consumer Interceptors
Key: KAFKA-3162
URL: https://issues.apache.org/jira/browse/KAFKA-3162
Project: Kafka
Issue Type: Bug
1 - 100 of 166 matches
Mail list logo