I think this is a good idea and your proposed changes look good.
I also think that this might be a good time to adopt KIP-185 (
https://cwiki.apache.org/confluence/display/KAFKA/KIP-185%3A+Make+exactly+once+in+order+delivery+per+partition+the+default+producer+setting),
and make the idempotent
+1 (non-binding)
On Tue, Oct 17, 2017 at 11:11 AM, Colin McCabe wrote:
> Hi all,
>
> I'd like to start the voting process for KIP-207:The Offsets which
> ListOffsetsResponse returns should monotonically increase even during a
> partition leader change.
>
> See
>
Apurva Mehta created KAFKA-6053:
---
Summary: NoSuchMethodError when creating ProducerRecord in upgrade
system tests
Key: KAFKA-6053
URL: https://issues.apache.org/jira/browse/KAFKA-6053
Project: Kafka
Thanks for the KIP Colin. That looks like a reasonable proposal.
On Thu, Oct 5, 2017 at 11:23 AM, Colin McCabe wrote:
> Hi all,
>
> I created a KIP for discussion about fixing a corner case in
> ListOffsetsResponse. Check it out at:
>
Apurva Mehta created KAFKA-6016:
---
Summary: Use the idempotent producer in the
reassign_partitions_test
Key: KAFKA-6016
URL: https://issues.apache.org/jira/browse/KAFKA-6016
Project: Kafka
Apurva Mehta created KAFKA-6015:
---
Summary: NPE in RecordAccumulator
Key: KAFKA-6015
URL: https://issues.apache.org/jira/browse/KAFKA-6015
Project: Kafka
Issue Type: Bug
Affects Versions
[
https://issues.apache.org/jira/browse/KAFKA-5552?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Apurva Mehta resolved KAFKA-5552.
-
Resolution: Cannot Reproduce
Fix Version/s: (was: 1.1.0
[
https://issues.apache.org/jira/browse/KAFKA-5865?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Apurva Mehta resolved KAFKA-5865.
-
Resolution: Fixed
This is fixed in 1.0.0 by the changes in
https://github.com/apache/kafka/pull
Apurva Mehta created KAFKA-5914:
---
Summary: Return MessageFormatVersion and MessageMaxBytes in
MetadataResponse
Key: KAFKA-5914
URL: https://issues.apache.org/jira/browse/KAFKA-5914
Project: Kafka
Apurva Mehta created KAFKA-5913:
---
Summary: Add RecordMetadataNotAvailableException to indicate that
ProduceResponse did not contain offset and timestamp information
Key: KAFKA-5913
URL: https://issues.apache.org
Apurva Mehta created KAFKA-5912:
---
Summary: Trogdor AgentTest.testAgentActivatesFaults is flaky
Key: KAFKA-5912
URL: https://issues.apache.org/jira/browse/KAFKA-5912
Project: Kafka
Issue Type
Hi Ted,
Unfortunately the jenkins.confluent.io address is no longer publicly
accessible.
Thanks,
Apurva
On Thu, Sep 14, 2017 at 7:35 PM, Ted Yu wrote:
> Hi,
> When I put the following in the address bar of Chrome:
>
>
Apurva Mehta created KAFKA-5897:
---
Summary: The producerId can be reset unnecessarily
Key: KAFKA-5897
URL: https://issues.apache.org/jira/browse/KAFKA-5897
Project: Kafka
Issue Type
The KIP has passed with three binding +1 votes (Guozhang, Ismael, Jason)
and no -1 or +0 votes.
Thanks to everyone for the feedback.
Apurva
On Mon, Sep 11, 2017 at 8:31 PM, Apurva Mehta <apu...@confluent.io> wrote:
> Hi Becket,
>
> You are right: the calculations are per par
Apurva Mehta created KAFKA-5888:
---
Summary: Transactions system test should check for message order
Key: KAFKA-5888
URL: https://issues.apache.org/jira/browse/KAFKA-5888
Project: Kafka
Issue
[
https://issues.apache.org/jira/browse/KAFKA-5621?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Apurva Mehta resolved KAFKA-5621.
-
Resolution: Won't Fix
> The producer should retry expired batches when retries are enab
about 35 MB of memory with 5 entries (142
> bytes) in cache. So it is probably still OK and it is more urgent to fix
> the upgrade path.
>
> Thanks,
>
> Jiangjie (Becket) Qin
>
>
>
>
>
> On Mon, Sep 11, 2017 at 4:13 PM, Apurva Mehta <apu...@confluent.i
because all
> the cached entries are the entries that hasn't been confirmed by the
> producer.
> 2. no magic number of 5 max.in.flight.requests.per.connection
> 3. bounded memory footprint on the cached sequence/timestamp/offset
> entries.
>
> Hope it's not too late to have the changes if that makes
Apurva Mehta created KAFKA-5870:
---
Summary: Idempotent producer: a producerId reset causes
undesirable behavior for inflight batches to other partitions
Key: KAFKA-5870
URL: https://issues.apache.org/jira/browse
.
Does anybody have any objections?
Thanks,
Apurva
On Thu, Sep 7, 2017 at 9:44 PM, Apurva Mehta <apu...@confluent.io> wrote:
> Thanks for the comments Ismael.
>
> I have gone ahead and incorporated all your suggestions in the KIP
> document. You convinced me on adding
Apurva Mehta created KAFKA-5865:
---
Summary: Expiring batches with idempotence enabled could cause
data loss.
Key: KAFKA-5865
URL: https://issues.apache.org/jira/browse/KAFKA-5865
Project: Kafka
; to MAX_INT and the existing default value of
> max.in.flight.request.per.connection==5, reordering becomes a possibility
> by default. To prevent reordering, set
> max.in.flight.request.per.connection==1.
>
> It does not hurt to mention it as it's a default behavior change?
>
> On 7 Sept
Thanks for the KIP Sumant, +1 from me.
That is the most exhaustive 'Rejected Alternatives' section that I have
seen :)
One minor point: In the compatibility section, your note on
'max.in.flight.requests.per.connection == 5' resulting in out of order
delivery is true irrespective of these
e that.
>
> Thanks,
> Ismael
>
> On Thu, Sep 7, 2017 at 9:51 PM, Apurva Mehta <apu...@confluent.io> wrote:
>
> > Hi,
> >
> > I'd like to start a vote for KIP-192:
> >
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > 192+%3A+Provide+cleaner+semantics+when+idempotence+is+enabled
> >
> > Thanks,
> > Apurva
> >
>
ble?
>
>
> Guozhang
>
>
> On Thu, Sep 7, 2017 at 2:25 PM, Jason Gustafson <ja...@confluent.io>
> wrote:
>
> > +1. Thanks for the KIP. One nit: we could use int8 to represent the
> message
> > format version. That is how it is represented in the messages
I agree with what Ismael said. Having both retries and delivery.timeout.ms
is confusing, and thus the goal is to not have a retries option at all once
idempotence is fully battle tested and has become the entrenched default.
Until that time, it makes sense to expire batch earlier than
Hi,
I'd like to start a vote for KIP-192:
https://cwiki.apache.org/confluence/display/KAFKA/KIP-192+%3A+Provide+cleaner+semantics+when+idempotence+is+enabled
Thanks,
Apurva
ly, maybe we really should be returning the message
> format version of each topic in the TopicMetadata response. A nice bonus of
> doing so is that it gives the producer the ability to craft the right
> format version ahead of time and avoid the need for conversion on the
> broker.
>
> Th
s
> > * with acks=minIsr, the remote time of each request will be min(lag of 2
> > followers)
> >
> > Whatever your latency distribution is for replication, for any given
> > remote time (say 100 ms), twice as many requests take longer than that
> time
> > with acks=all vs a
rs
>
> On Wed, Aug 30, 2017 at 3:47 PM, Apurva Mehta <apu...@confluent.io> wrote:
>
> > Thanks Ismael and Jason, I filed a separate KIP to solve the problems
> > identified through this discussion. I also incorporated Jason's comments
> in
> > that document:
>
ke sense
> to
> > include that here so that the OutOfOrderSequence error is unambiguous.
> >
> > Finally, do you plan to roll these proposals into the current KIP or do
> > them separately? Probably makes sense to combine them since they both
> > re
Hi,
In the discussion of KIP-185 (enable idempotence by default), we discovered
some shortcomings of the existing idempotent producer implementation.
Fixing these issues requires changes to the ProduceRequest and
ProduceResponse protocols as well as changes to the values of the
[
https://issues.apache.org/jira/browse/KAFKA-5640?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Apurva Mehta resolved KAFKA-5640.
-
Resolution: Duplicate
This is a dup of https://issues.apache.org/jira/browse/KAFKA-5796
> L
Apurva Mehta created KAFKA-5796:
---
Summary: Understand performance implications of acks=all and
potential ways to reduce it
Key: KAFKA-5796
URL: https://issues.apache.org/jira/browse/KAFKA-5796
Project
Apurva Mehta created KAFKA-5795:
---
Summary: Make the idempotent producer the default producer setting
Key: KAFKA-5795
URL: https://issues.apache.org/jira/browse/KAFKA-5795
Project: Kafka
Issue
Apurva Mehta created KAFKA-5794:
---
Summary: Introduce new idempotence mode to gracefully deal with
topics on the older message format
Key: KAFKA-5794
URL: https://issues.apache.org/jira/browse/KAFKA-5794
Apurva Mehta created KAFKA-5793:
---
Summary: Tighten up situations where OutOfOrderSequence may be
returned
Key: KAFKA-5793
URL: https://issues.apache.org/jira/browse/KAFKA-5793
Project: Kafka
Apurva Mehta created KAFKA-5792:
---
Summary: Transient failure in
KafkaAdminClientTest.testHandleTimeout
Key: KAFKA-5792
URL: https://issues.apache.org/jira/browse/KAFKA-5792
Project: Kafka
Hi Pavel,
I believe that you have been added as a contributor on the Wiki. You can
move your KIP to the correct space now. Note that you will need to reassign
the KIP number since 190 is already taken.
Thanks,
Apurva
On Mon, Aug 21, 2017 at 10:52 AM, Apurva Mehta <apu...@confluent.io>
Hi Pavel,
Thanks for the KIP. You need to be added to the contributors list before
you can modify the Kafka wiki space. One of the committers will add you and
let you know when it has happened. After that, you can create the KIP in
the right space.
Thanks,
Apurva
On Fri, Aug 18, 2017 at 12:55
Thanks Jason and Ismael.
The message format problem is an acute one: if we enable idempotence by
default, the UnsupportedVersionException when writing to topics with the
older message format would mean that our prescribed upgrade steps would not
work. I have detailed the problems and the
Is that correct?
>
> Overall I'm definitely supportive of making idempotence the default
> eventually, but I think it might be a tad premature now.
>
> Thanks,
> Jason
>
> On Wed, Aug 16, 2017 at 8:58 PM, Apurva Mehta <apu...@confluent.io> wrote:
>
> >
is
> turned
> > > on,
> > > > but this is not enforced at the broker but relying at the client
> side's
> > > > sanity. So other implementations of the client that may not obey this
> > may
> > > > likely break the broker code. If w
that may not obey this may
> > likely break the broker code. If we do enforce this we'd better enforce
> it
> > at the broker side. Also, I'm wondering if we have considered the
> approach
> > for brokers to read the logs in order to get the starting offset when it
> > doe
Hello,
I just want to summarize where we are in this discussion
There are two major points of contention: should we have acks=1 or acsk=all
by default? and how to cap max.in.flight.requests.per.connection?
1) acks=1 vs acks=all1
Here are the tradeoffs of each:
If you have
I think the question of the default broker level configs is a good one. I
don't think we need to touch the min.isr config or the replication factor
to satisfy 'exactly-once' going by the definition laid out earlier. On the
broker side, I think the only thing we should change is to disable unclean
here is no available memory in buffer then report
> to me immidiately that there is not enough memory.
>
>
> 2017-08-11 21:04 GMT+03:00 Apurva Mehta <apu...@confluent.io>:
>
> > What precise use case do you have in mind? If you don't have cluster
> > meta
+1,
Thanks,
Apurva
On Fri, Aug 11, 2017 at 3:02 PM, Bill Bejeck wrote:
> +1
>
> Thanks,
> Bill
>
> On Fri, Aug 11, 2017 at 6:00 PM, Colin McCabe wrote:
>
> > Hi all,
> >
> > I think it's a good time to vote on KIP-180. It adds a helpful new
> > metric
s to
> send, we need to have max.in.flight.requests ~ 20 in order to fully utilize
> the network bandwidth. When the requests are smaller, we will need to
> pipeline more requests.
>
> Thanks,
>
> Jiangjie (Becket) Qin
>
>
>
>
> On Thu, Aug 10, 2017 at 10:4
What precise use case do you have in mind? If you don't have cluster
metadata, you can't send the requests anyway. If you want to bound your
memory and run out of it, that means that you are not able to send data for
some reason.
The best you can do in both cases is to drop old messages from the
ong benefit of doing so (e.g. exactly-once).
>
> Thanks,
> Dong
>
>
>
>
> On Wed, Aug 9, 2017 at 10:43 PM, Apurva Mehta <apu...@confluent.io> wrote:
>
> > Thanks for your email Becket.
> >
> > Your observations around using acks=1 and acks=-1 are co
g a cap on the max.in.flight.requests. It seems
> that on some long RTT link, sending more requests in the pipeline would be
> the only way to keep the latency to be close to RTT.
>
> Thanks,
>
> Jiangjie (Becket) Qin
>
>
> On Wed, Aug 9, 2017 at 11:28 AM, Apurva Mehta <apu...@c
> > There seems to be no relationship with cluster metadata availability or
> > staleness. Expiry is just based on the time since the batch has been
> ready.
> > Please correct me if I am wrong.
> >
>
> I was not very specific about where we do expiration. I glossed over some
> details because
Thanks for the KIP. +1 from me.
On Tue, Aug 8, 2017 at 5:24 PM, Ewen Cheslack-Postava
wrote:
> Hi all,
>
> I posted a simple new KIP for a problem we see with a lot of users:
> KIP-186: Increase offsets retention default to 7 days
>
>
Thanks for the comments Ismael and Jason.
Regarding the OutOfOrderSequenceException, it is more likely when you
enable idempotence and have acks=1, simply because you have a greater
probability of losing acknowledged data with acks=1, and the error code
indicates that.
The particular scenario is
oned above, this means that each acknowledged
send should result in exactly one copy of the message in the log. With
acks=1, we can only ever have at-most once delivery, ie. an acknowledged
send could result in 0 copies of the message in the log. Please let me know
if I have missed something.
Th
Hi,
I've put together a new KIP which proposes to ship Kafka with its strongest
delivery guarantees by default.
We currently ship with at most once semantics and don't provide any
ordering guarantees per partition. The proposal is is to provide exactly
once in order delivery per partition by
Hi Colin,
The KIP looks good to me. In your latest proposal, the change of state
would be captured as followed in the metrics for groups using Kafka for
membership management:
PreparingRebalance -> CompletingRebalance -> Stable -> Dead?
If a group is just being used to store offsets, then it is
Responses inline:
On Mon, Aug 7, 2017 at 9:37 AM, Sumant Tambe wrote:
>
> >
> > However, one thing which has not come out of the JIRA discussion is the
> > actual use cases for batch expiry.
>
> There are two usecases I can think of for batch expiry mechanism
> irrespective
If we are going to have a separate configuration for expiry, I prefer my
proposal of max.message.delivery.wait.ms and its semantics.
However, one thing which has not come out of the JIRA discussion is the
actual use cases for batch expiry. Also, the KIP document states the
following:
*The per
Apurva Mehta created KAFKA-5679:
---
Summary: Add logging to distinguish between internally and
externally initiated shutdown of Kafka
Key: KAFKA-5679
URL: https://issues.apache.org/jira/browse/KAFKA-5679
Apurva Mehta created KAFKA-5663:
---
Summary: LogDirFailureTest system test fails
Key: KAFKA-5663
URL: https://issues.apache.org/jira/browse/KAFKA-5663
Project: Kafka
Issue Type: Bug
Apurva Mehta created KAFKA-5662:
---
Summary: We should be able to specify min.insync.replicas for the
__consumer_offsets topic
Key: KAFKA-5662
URL: https://issues.apache.org/jira/browse/KAFKA-5662
Apurva Mehta created KAFKA-5661:
---
Summary: Develop and understanding of how to tune transactions for
optimal performance
Key: KAFKA-5661
URL: https://issues.apache.org/jira/browse/KAFKA-5661
Project
Apurva Mehta created KAFKA-5640:
---
Summary: Look into making acks=all the default setting.
Key: KAFKA-5640
URL: https://issues.apache.org/jira/browse/KAFKA-5640
Project: Kafka
Issue Type: Sub
Hi Ovidu,
The see-saw behavior is inevitable with linux when you have concurrent
reads and writes. However, tuning the following two settings may help
achieve more stable performance (from Jay's link):
> *dirty_ratio*Defines a percentage value. Writeout of dirty data begins
> (via *pdflush*)
Apurva Mehta created KAFKA-5621:
---
Summary: The producer should retry expired batches when retries
are enabled
Key: KAFKA-5621
URL: https://issues.apache.org/jira/browse/KAFKA-5621
Project: Kafka
Apurva Mehta created KAFKA-5610:
---
Summary: KafkaApis.handleWriteTxnMarkerRequest can return
UNSUPPORTED_FOR_MESSAGE_FORMAT error on partition emigration
Key: KAFKA-5610
URL: https://issues.apache.org/jira/browse
Apurva Mehta created KAFKA-5604:
---
Summary: All producer methods should raise
`ProducerFencedException` after the first time.
Key: KAFKA-5604
URL: https://issues.apache.org/jira/browse/KAFKA-5604
Apurva Mehta created KAFKA-5543:
---
Summary: We don't remove the LastStableOffsetLag metric when a
partition is moved away from a broker
Key: KAFKA-5543
URL: https://issues.apache.org/jira/browse/KAFKA-5543
Apurva Mehta created KAFKA-5494:
---
Summary: Idempotent producer should not require
max.in.flight.requests.per.connection=1 and acks=all
Key: KAFKA-5494
URL: https://issues.apache.org/jira/browse/KAFKA-5494
Apurva Mehta created KAFKA-5491:
---
Summary: The ProducerPerformance tool should support transactions
Key: KAFKA-5491
URL: https://issues.apache.org/jira/browse/KAFKA-5491
Project: Kafka
Issue
Hi Tom,
I actually made modifications to the produce performance tool to do real
transactions earlier this week as part of our benchmarking (results
published here: bit.ly/kafka-eos-perf). I just submitted that patch here:
https://github.com/apache/kafka/pull/3400/files
I think my version is
Apurva Mehta created KAFKA-5482:
---
Summary: A CONCURRENT_TRANASCTIONS error for the first
AddPartitionsToTxn request slows down transactions significantly
Key: KAFKA-5482
URL: https://issues.apache.org/jira/browse
Apurva Mehta created KAFKA-5477:
---
Summary: TransactionalProducer sleeps unnecessarily long during
back to back transactions
Key: KAFKA-5477
URL: https://issues.apache.org/jira/browse/KAFKA-5477
Project
Apurva Mehta created KAFKA-5457:
---
Summary: RecordAccumulator.hasRoomfor doesn't take into account
the headers while computing available space
Key: KAFKA-5457
URL: https://issues.apache.org/jira/browse/KAFKA-5457
[
https://issues.apache.org/jira/browse/KAFKA-5449?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Apurva Mehta updated KAFKA-5449:
Priority: Blocker (was: Major)
> Flaky t
[
https://issues.apache.org/jira/browse/KAFKA-5449?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Apurva Mehta reassigned KAFKA-5449:
---
Assignee: Jason Gustafson (was: Apurva Mehta)
> Flaky t
Apurva Mehta created KAFKA-5455:
---
Summary: Update java docs for consumer and producer to be up to
date for EOS
Key: KAFKA-5455
URL: https://issues.apache.org/jira/browse/KAFKA-5455
Project: Kafka
[
https://issues.apache.org/jira/browse/KAFKA-5455?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Apurva Mehta updated KAFKA-5455:
Labels: exactly-once (was: )
> Update java docs for consumer and producer to be up to date for
[
https://issues.apache.org/jira/browse/KAFKA-5449?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16049800#comment-16049800
]
Apurva Mehta commented on KAFKA-5449:
-
With the latest patch, it appears that the producer which
[
https://issues.apache.org/jira/browse/KAFKA-5449?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Apurva Mehta reassigned KAFKA-5449:
---
Assignee: Apurva Mehta
> Flaky t
[
https://issues.apache.org/jira/browse/KAFKA-5020?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Apurva Mehta reassigned KAFKA-5020:
---
Assignee: Apurva Mehta
> Update protocol documentation to mention message format
[
https://issues.apache.org/jira/browse/KAFKA-5317?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Apurva Mehta resolved KAFKA-5317.
-
Resolution: Fixed
> Update KIP-98 to reflect changes during implementat
[
https://issues.apache.org/jira/browse/KAFKA-5317?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16048515#comment-16048515
]
Apurva Mehta commented on KAFKA-5317:
-
I updated both the wiki and the google doc to reflect
[
https://issues.apache.org/jira/browse/KAFKA-5052?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Apurva Mehta updated KAFKA-5052:
Description:
* This is a follow up to : https://issues.apache.org/jira/browse/KAFKA-3527
We
[
https://issues.apache.org/jira/browse/KAFKA-5441?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Apurva Mehta updated KAFKA-5441:
Labels: exactly-once (was: )
> Fix transaction marker grouping by producerId in
[
https://issues.apache.org/jira/browse/KAFKA-5317?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Apurva Mehta reassigned KAFKA-5317:
---
Assignee: Apurva Mehta
> Update KIP-98 to reflect changes during implementat
[
https://issues.apache.org/jira/browse/KAFKA-5438?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Apurva Mehta updated KAFKA-5438:
Labels: exactly-once (was: )
> UnsupportedOperationException in WriteTxnMarkers hand
[
https://issues.apache.org/jira/browse/KAFKA-5436?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Apurva Mehta reassigned KAFKA-5436:
---
Assignee: Jason Gustafson (was: Apurva Mehta)
> NullPointerException when loading produ
[
https://issues.apache.org/jira/browse/KAFKA-5436?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Apurva Mehta updated KAFKA-5436:
Issue Type: Sub-task (was: Bug)
Parent: KAFKA-4815
> NullPointerException when load
Apurva Mehta created KAFKA-5436:
---
Summary: NullPointerException when loading producer snapshot
Key: KAFKA-5436
URL: https://issues.apache.org/jira/browse/KAFKA-5436
Project: Kafka
Issue Type
[
https://issues.apache.org/jira/browse/KAFKA-5428?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Apurva Mehta updated KAFKA-5428:
Labels: exactly-once (was: )
> Transactional producer aborts batches incorrectly in aborta
[
https://issues.apache.org/jira/browse/KAFKA-5427?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Apurva Mehta updated KAFKA-5427:
Labels: exactly-once (was: )
> Transactional producer cannot find coordinator when trying to ab
[
https://issues.apache.org/jira/browse/KAFKA-5342?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Apurva Mehta updated KAFKA-5342:
Fix Version/s: (was: 0.11.0.0)
0.11.0.1
> Distinguish abortable failu
[
https://issues.apache.org/jira/browse/KAFKA-5286?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Apurva Mehta updated KAFKA-5286:
Fix Version/s: (was: 0.11.0.0)
0.11.0.1
> Producer should await transact
[
https://issues.apache.org/jira/browse/KAFKA-5415?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Apurva Mehta updated KAFKA-5415:
Status: Patch Available (was: Open)
> TransactionCoordinator doesn't complete transit
[
https://issues.apache.org/jira/browse/KAFKA-5415?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16045162#comment-16045162
]
Apurva Mehta edited comment on KAFKA-5415 at 6/9/17 10:48 PM:
--
The last
[
https://issues.apache.org/jira/browse/KAFKA-5415?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16045162#comment-16045162
]
Apurva Mehta commented on KAFKA-5415:
-
The last successful metadata update was the following
[
https://issues.apache.org/jira/browse/KAFKA-5415?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16045151#comment-16045151
]
Apurva Mehta commented on KAFKA-5415:
-
The problem here seem to be this check:
https://github.com
1 - 100 of 433 matches
Mail list logo