Re: [DISCUSS] KIP-1059: Enable the Producer flush() method to clear the latest send() error

2024-07-24 Thread Artem Livshits
Hi Greg,

I'm sorry if anything I said sounded like I'm trying to minimize the
concerns that's definitely not my intention.  My background is in
databases, and I share your concerns of muddying transaction semantics
(which is already, as you pointed out, is different in Kafka).

If calling some kind of record pre-validation method (to be called
explicitly before send) is acceptable to KStreams that would be my
preference as well, because that expresses the intent of the desired
functionality, IMO.

One suggestion I would like to make is to not name it "prepare", as there
will be a prepareTransaction method once
https://cwiki.apache.org/confluence/display/KAFKA/KIP-939%3A+Support+Participation+in+2PC
is implemented and could be confusing to have 2 methods named "prepare"
that mean completely different things.

Maybe name it valiateRecord?

-Artem

On Fri, Jul 19, 2024 at 4:19 PM Greg Harris 
wrote:

> Hi Artem and Matthias,
>
> > On the other hand, the effort to prove that
> > keeping all records in memory won't break some scenarios (and generally
> > breaking one is enough to cause a lot of pain) seems to be significantly
> > higher than to prove that setting some flag in some API has pretty much 0
> > chance of regression
>
> > in the end, why buffer records twice?
>
> > This way we don't
> > ignore the error, we're just changing the method they are delivered.
>
> > Very clean semantics
> > which should also address the concern of "non-atomic tx"
>
> I feel like my concerns are being minimized instead of being addressed
> in this discussion, and if that's because I'm not expressing them clearly,
> I apologize.
>
> Many users come to Kafka with prior expectations, especially when we use
> industry-standard terminology like 'Exactly Once Semantics",
> "Transactions", "Commit", "Abort". Of course Kafka isn't an ACID-compliant
> database, but users will evaluate, discuss, and develop applications with
> Kafka through the lens of the ACID principles, because that is the
> framework most commonly applied to transactional semantics.
> The original design of KIP-98 [1] explicitly mentions atomic commits (with
> the same meaning as the A in ACID) as the primary abstraction being added
> (reproduced here):
>
> > At the core, transactional guarantees enable applications to produce to
> multiple TopicPartitions atomically, ie. all writes to these
> TopicPartitions will succeed or fail as a unit.
> > Further, since consumer progress is recorded as a write to the offsets
> topic, the above capability is leveraged to enable applications to batch
> consumed and produced messages into a single atomic unit, ie. a set of
> messages may be considered consumed only if the entire
> ‘consume-transform-produce’ executed in its entirety.
>
> I think it's important to say that to a user, "writes" really means "send()
> and commitOffsets() calls", not literal produce requests to Kafka brokers,
> and "consume-transform-produce" really means "poll(), transform, send()".
> This is because to a user, the implementation within poll() and send() and
> the broker are none of their concern, and are intended to be within the
> abstraction.
> When I say that this feature is a non-atomic commit, I mean that this
> feature does not fit the above description, and breaks the transaction
> abstraction in a meaningful way. No longer do all writes succeed or fail as
> a unit, some failures are permitted to drop data. No longer must a
> consume-transform-produce cycle be executed in its entirety, some parts may
> be left incomplete.
> This means that this method will be difficult to define ("which exceptions
> are covered?"), difficult to document ("how do we explain
> 'not-really-atomic commits' clearly and unambiguously to a potential
> user?"), and difficult to compose ("if someone turns this option on, how
> does that affect delivery guarantees and opportunities for bugs in
> upper layers?").
> Users currently rely heavily on analogies to other database systems to make
> sense of Kafka's transactions, and we need to use that to our benefit,
> rather than designing in spite of it being true.
>
> However this atomicity guarantee isn't always desirable, as evidenced by
> the original bug report [2]. If you're interacting with a website form for
> example, and a database transaction fails because one of your strings is
> oversize, you don't need to re-input all of your form responses from
> scratch, as there is an application layer/browser in-between to preserve
> the state and retry the transaction.
> And while you could make a convenience/performance/etc argument in that
> situation ("The database should truncate/null-out the oversize string") and
> modern databases often have very expressive DML that would permit such a
> behavior (try-catch, etc), the End-to-End arguments [3] make me believe
> that is a bad design and should be discouraged.
> To that end, I was suggesting ways to push this farther and farther up the
> stack, such as performing r

Re: [DISCUSS] KIP-1059: Enable the Producer flush() method to clear the latest send() error

2024-07-23 Thread Andrew Schofield
Hi,
I’m trying to understand the crux of the discussion here.

In systems I’m used to, it’s entirely allowed to attempt to add an operation to 
a transaction,
have that operation unambiguously fail, and continue with the transaction safe 
in the
knowledge that the failed operation didn’t make it, but the transaction is 
intact. The
“unambiguously” part of this is key. If the application doesn’t really know 
what’s in the
transaction, it’s not safe to commit it. It is of course always safe to roll it 
back.

So, I think the core question is whether this KIP can achieve this for Kafka for
Producer.send() operations. Personally, I’m pretty happy with the Producer 
putting
the transaction into error state unless it is safely handling a small number of 
errors
for situations in which it’s sure that the transaction is intact.

I’m comfortable with an additional method, such as prepare(), so that the 
current
behaviour of send() is retained wherein a failed operation always puts the
transaction into an error state so it cannot be committed. This is probably more
robust in terms of unambiguous behaviour than overloading send() or flush().

Thanks,
Andrew

> On 23 Jul 2024, at 00:04, Matthias J. Sax  wrote:
>
> Thanks Greg,
>
> Apologize if we gave the impression that we did not try to address your 
> concerns. It was not my intention to just minimize them. Of course, I just 
> don't see it your way, but that's ok. We don't have to agree. That's why we 
> have a DISCUSS thread to begin with.
>
> I don't know the producer internal well enough to judge, how easy/difficult 
> it would be to implement a `prepare(..)` method as proposed. If it works, I 
> won't object to go down this route. Happy to disagree and commit.
>
>
> I just don't see why `prepare()` is semantically cleaner/better as you say, 
> compared to a `send()` which would throw an exception directly? But this 
> might be personal preference / judgment, and it might not benefit this KIP to 
> argue about it further.
>
>
> I am wondering about interleaved calls of `prepare()` and regular `send()` 
> though, especially given that the producer is thread-safe. Maybe there is 
> nothing to worry about (as said, I don't know the producer internal well 
> enough), but if this would cause issues, it might not be the best way forward.
>
>
> In the end it's Alieh's KIP, and it seems adding `prepare(...)` will enlarge 
> to scope of the KIP. So it's her call if she wants to go down this path or 
> not.
>
>
> -Matthias
>
>
> On 7/22/24 12:30 PM, Greg Harris wrote:
>> Hi Alieh,
>> Yes, I think you understand my intent for the prepare() method.
>> Thanks,
>> Greg
>> On Mon, Jul 22, 2024 at 2:54 AM Alieh Saeedi 
>> wrote:
>>> Hi Greg,
>>>
>>>
>>> I appreciate your concerns and comprehensive answer.
>>>
>>>
>>> I am not sure whether I fully understood what you meant or not. You mean,
>>> at the end, the user can go for one of the following scenarios: Either
>>>
>>> 1) `beginTxn()` and `send(record)` and `commitTxn()`  or
>>>
>>> 2) `beginTxn()` and `prepare(record)` and `send(prepared_record)` and
>>> `commitTxn()` ?
>>>
>>>
>>> Of course, the `send` in scenario 1 is different from the one in scenario
>>> 2, since a part of the second one 's job has been done during
>>> `prepare()`ing.
>>>
>>>
>>> Cheers,
>>>
>>> Alieh
>>>
>>> On Sat, Jul 20, 2024 at 1:20 AM Greg Harris 
>>> wrote:
>>>
 Hi Artem and Matthias,

> On the other hand, the effort to prove that
> keeping all records in memory won't break some scenarios (and generally
> breaking one is enough to cause a lot of pain) seems to be
>>> significantly
> higher than to prove that setting some flag in some API has pretty
>>> much 0
> chance of regression

> in the end, why buffer records twice?

> This way we don't
> ignore the error, we're just changing the method they are delivered.

> Very clean semantics
> which should also address the concern of "non-atomic tx"

 I feel like my concerns are being minimized instead of being addressed
 in this discussion, and if that's because I'm not expressing them
>>> clearly,
 I apologize.

 Many users come to Kafka with prior expectations, especially when we use
 industry-standard terminology like 'Exactly Once Semantics",
 "Transactions", "Commit", "Abort". Of course Kafka isn't an
>>> ACID-compliant
 database, but users will evaluate, discuss, and develop applications with
 Kafka through the lens of the ACID principles, because that is the
 framework most commonly applied to transactional semantics.
 The original design of KIP-98 [1] explicitly mentions atomic commits
>>> (with
 the same meaning as the A in ACID) as the primary abstraction being added
 (reproduced here):

> At the core, transactional guarantees enable applications to produce to
 multiple TopicPartitions atomically, ie. all writes to these
 TopicPartitions will succeed or fail as 

Re: [DISCUSS] KIP-1059: Enable the Producer flush() method to clear the latest send() error

2024-07-22 Thread Matthias J. Sax

Thanks Greg,

Apologize if we gave the impression that we did not try to address your 
concerns. It was not my intention to just minimize them. Of course, I 
just don't see it your way, but that's ok. We don't have to agree. 
That's why we have a DISCUSS thread to begin with.


I don't know the producer internal well enough to judge, how 
easy/difficult it would be to implement a `prepare(..)` method as 
proposed. If it works, I won't object to go down this route. Happy to 
disagree and commit.



I just don't see why `prepare()` is semantically cleaner/better as you 
say, compared to a `send()` which would throw an exception directly? But 
this might be personal preference / judgment, and it might not benefit 
this KIP to argue about it further.



I am wondering about interleaved calls of `prepare()` and regular 
`send()` though, especially given that the producer is thread-safe. 
Maybe there is nothing to worry about (as said, I don't know the 
producer internal well enough), but if this would cause issues, it might 
not be the best way forward.



In the end it's Alieh's KIP, and it seems adding `prepare(...)` will 
enlarge to scope of the KIP. So it's her call if she wants to go down 
this path or not.



-Matthias


On 7/22/24 12:30 PM, Greg Harris wrote:

Hi Alieh,

Yes, I think you understand my intent for the prepare() method.

Thanks,
Greg

On Mon, Jul 22, 2024 at 2:54 AM Alieh Saeedi 
wrote:


Hi Greg,


I appreciate your concerns and comprehensive answer.


I am not sure whether I fully understood what you meant or not. You mean,
at the end, the user can go for one of the following scenarios: Either

1) `beginTxn()` and `send(record)` and `commitTxn()`  or

2) `beginTxn()` and `prepare(record)` and `send(prepared_record)` and
`commitTxn()` ?


Of course, the `send` in scenario 1 is different from the one in scenario
2, since a part of the second one 's job has been done during
`prepare()`ing.


Cheers,

Alieh

On Sat, Jul 20, 2024 at 1:20 AM Greg Harris 
wrote:


Hi Artem and Matthias,


On the other hand, the effort to prove that
keeping all records in memory won't break some scenarios (and generally
breaking one is enough to cause a lot of pain) seems to be

significantly

higher than to prove that setting some flag in some API has pretty

much 0

chance of regression



in the end, why buffer records twice?



This way we don't
ignore the error, we're just changing the method they are delivered.



Very clean semantics
which should also address the concern of "non-atomic tx"


I feel like my concerns are being minimized instead of being addressed
in this discussion, and if that's because I'm not expressing them

clearly,

I apologize.

Many users come to Kafka with prior expectations, especially when we use
industry-standard terminology like 'Exactly Once Semantics",
"Transactions", "Commit", "Abort". Of course Kafka isn't an

ACID-compliant

database, but users will evaluate, discuss, and develop applications with
Kafka through the lens of the ACID principles, because that is the
framework most commonly applied to transactional semantics.
The original design of KIP-98 [1] explicitly mentions atomic commits

(with

the same meaning as the A in ACID) as the primary abstraction being added
(reproduced here):


At the core, transactional guarantees enable applications to produce to

multiple TopicPartitions atomically, ie. all writes to these
TopicPartitions will succeed or fail as a unit.

Further, since consumer progress is recorded as a write to the offsets

topic, the above capability is leveraged to enable applications to batch
consumed and produced messages into a single atomic unit, ie. a set of
messages may be considered consumed only if the entire
‘consume-transform-produce’ executed in its entirety.

I think it's important to say that to a user, "writes" really means

"send()

and commitOffsets() calls", not literal produce requests to Kafka

brokers,

and "consume-transform-produce" really means "poll(), transform, send()".
This is because to a user, the implementation within poll() and send()

and

the broker are none of their concern, and are intended to be within the
abstraction.
When I say that this feature is a non-atomic commit, I mean that this
feature does not fit the above description, and breaks the transaction
abstraction in a meaningful way. No longer do all writes succeed or fail

as

a unit, some failures are permitted to drop data. No longer must a
consume-transform-produce cycle be executed in its entirety, some parts

may

be left incomplete.
This means that this method will be difficult to define ("which

exceptions

are covered?"), difficult to document ("how do we explain
'not-really-atomic commits' clearly and unambiguously to a potential
user?"), and difficult to compose ("if someone turns this option on, how
does that affect delivery guarantees and opportunities for bugs in
upper layers?").
Users currently rely heavily on analogies to other database systems to

Re: [DISCUSS] KIP-1059: Enable the Producer flush() method to clear the latest send() error

2024-07-22 Thread Greg Harris
Hi Alieh,

Yes, I think you understand my intent for the prepare() method.

Thanks,
Greg

On Mon, Jul 22, 2024 at 2:54 AM Alieh Saeedi 
wrote:

> Hi Greg,
>
>
> I appreciate your concerns and comprehensive answer.
>
>
> I am not sure whether I fully understood what you meant or not. You mean,
> at the end, the user can go for one of the following scenarios: Either
>
> 1) `beginTxn()` and `send(record)` and `commitTxn()`  or
>
> 2) `beginTxn()` and `prepare(record)` and `send(prepared_record)` and
> `commitTxn()` ?
>
>
> Of course, the `send` in scenario 1 is different from the one in scenario
> 2, since a part of the second one 's job has been done during
> `prepare()`ing.
>
>
> Cheers,
>
> Alieh
>
> On Sat, Jul 20, 2024 at 1:20 AM Greg Harris 
> wrote:
>
> > Hi Artem and Matthias,
> >
> > > On the other hand, the effort to prove that
> > > keeping all records in memory won't break some scenarios (and generally
> > > breaking one is enough to cause a lot of pain) seems to be
> significantly
> > > higher than to prove that setting some flag in some API has pretty
> much 0
> > > chance of regression
> >
> > > in the end, why buffer records twice?
> >
> > > This way we don't
> > > ignore the error, we're just changing the method they are delivered.
> >
> > > Very clean semantics
> > > which should also address the concern of "non-atomic tx"
> >
> > I feel like my concerns are being minimized instead of being addressed
> > in this discussion, and if that's because I'm not expressing them
> clearly,
> > I apologize.
> >
> > Many users come to Kafka with prior expectations, especially when we use
> > industry-standard terminology like 'Exactly Once Semantics",
> > "Transactions", "Commit", "Abort". Of course Kafka isn't an
> ACID-compliant
> > database, but users will evaluate, discuss, and develop applications with
> > Kafka through the lens of the ACID principles, because that is the
> > framework most commonly applied to transactional semantics.
> > The original design of KIP-98 [1] explicitly mentions atomic commits
> (with
> > the same meaning as the A in ACID) as the primary abstraction being added
> > (reproduced here):
> >
> > > At the core, transactional guarantees enable applications to produce to
> > multiple TopicPartitions atomically, ie. all writes to these
> > TopicPartitions will succeed or fail as a unit.
> > > Further, since consumer progress is recorded as a write to the offsets
> > topic, the above capability is leveraged to enable applications to batch
> > consumed and produced messages into a single atomic unit, ie. a set of
> > messages may be considered consumed only if the entire
> > ‘consume-transform-produce’ executed in its entirety.
> >
> > I think it's important to say that to a user, "writes" really means
> "send()
> > and commitOffsets() calls", not literal produce requests to Kafka
> brokers,
> > and "consume-transform-produce" really means "poll(), transform, send()".
> > This is because to a user, the implementation within poll() and send()
> and
> > the broker are none of their concern, and are intended to be within the
> > abstraction.
> > When I say that this feature is a non-atomic commit, I mean that this
> > feature does not fit the above description, and breaks the transaction
> > abstraction in a meaningful way. No longer do all writes succeed or fail
> as
> > a unit, some failures are permitted to drop data. No longer must a
> > consume-transform-produce cycle be executed in its entirety, some parts
> may
> > be left incomplete.
> > This means that this method will be difficult to define ("which
> exceptions
> > are covered?"), difficult to document ("how do we explain
> > 'not-really-atomic commits' clearly and unambiguously to a potential
> > user?"), and difficult to compose ("if someone turns this option on, how
> > does that affect delivery guarantees and opportunities for bugs in
> > upper layers?").
> > Users currently rely heavily on analogies to other database systems to
> make
> > sense of Kafka's transactions, and we need to use that to our benefit,
> > rather than designing in spite of it being true.
> >
> > However this atomicity guarantee isn't always desirable, as evidenced by
> > the original bug report [2]. If you're interacting with a website form
> for
> > example, and a database transaction fails because one of your strings is
> > oversize, you don't need to re-input all of your form responses from
> > scratch, as there is an application layer/browser in-between to preserve
> > the state and retry the transaction.
> > And while you could make a convenience/performance/etc argument in that
> > situation ("The database should truncate/null-out the oversize string")
> and
> > modern databases often have very expressive DML that would permit such a
> > behavior (try-catch, etc), the End-to-End arguments [3] make me believe
> > that is a bad design and should be discouraged.
> > To that end, I was suggesting ways to push this farther and farther up
> the

Re: [DISCUSS] KIP-1059: Enable the Producer flush() method to clear the latest send() error

2024-07-22 Thread Alieh Saeedi
Hi Greg,


I appreciate your concerns and comprehensive answer.


I am not sure whether I fully understood what you meant or not. You mean,
at the end, the user can go for one of the following scenarios: Either

1) `beginTxn()` and `send(record)` and `commitTxn()`  or

2) `beginTxn()` and `prepare(record)` and `send(prepared_record)` and
`commitTxn()` ?


Of course, the `send` in scenario 1 is different from the one in scenario
2, since a part of the second one 's job has been done during
`prepare()`ing.


Cheers,

Alieh

On Sat, Jul 20, 2024 at 1:20 AM Greg Harris 
wrote:

> Hi Artem and Matthias,
>
> > On the other hand, the effort to prove that
> > keeping all records in memory won't break some scenarios (and generally
> > breaking one is enough to cause a lot of pain) seems to be significantly
> > higher than to prove that setting some flag in some API has pretty much 0
> > chance of regression
>
> > in the end, why buffer records twice?
>
> > This way we don't
> > ignore the error, we're just changing the method they are delivered.
>
> > Very clean semantics
> > which should also address the concern of "non-atomic tx"
>
> I feel like my concerns are being minimized instead of being addressed
> in this discussion, and if that's because I'm not expressing them clearly,
> I apologize.
>
> Many users come to Kafka with prior expectations, especially when we use
> industry-standard terminology like 'Exactly Once Semantics",
> "Transactions", "Commit", "Abort". Of course Kafka isn't an ACID-compliant
> database, but users will evaluate, discuss, and develop applications with
> Kafka through the lens of the ACID principles, because that is the
> framework most commonly applied to transactional semantics.
> The original design of KIP-98 [1] explicitly mentions atomic commits (with
> the same meaning as the A in ACID) as the primary abstraction being added
> (reproduced here):
>
> > At the core, transactional guarantees enable applications to produce to
> multiple TopicPartitions atomically, ie. all writes to these
> TopicPartitions will succeed or fail as a unit.
> > Further, since consumer progress is recorded as a write to the offsets
> topic, the above capability is leveraged to enable applications to batch
> consumed and produced messages into a single atomic unit, ie. a set of
> messages may be considered consumed only if the entire
> ‘consume-transform-produce’ executed in its entirety.
>
> I think it's important to say that to a user, "writes" really means "send()
> and commitOffsets() calls", not literal produce requests to Kafka brokers,
> and "consume-transform-produce" really means "poll(), transform, send()".
> This is because to a user, the implementation within poll() and send() and
> the broker are none of their concern, and are intended to be within the
> abstraction.
> When I say that this feature is a non-atomic commit, I mean that this
> feature does not fit the above description, and breaks the transaction
> abstraction in a meaningful way. No longer do all writes succeed or fail as
> a unit, some failures are permitted to drop data. No longer must a
> consume-transform-produce cycle be executed in its entirety, some parts may
> be left incomplete.
> This means that this method will be difficult to define ("which exceptions
> are covered?"), difficult to document ("how do we explain
> 'not-really-atomic commits' clearly and unambiguously to a potential
> user?"), and difficult to compose ("if someone turns this option on, how
> does that affect delivery guarantees and opportunities for bugs in
> upper layers?").
> Users currently rely heavily on analogies to other database systems to make
> sense of Kafka's transactions, and we need to use that to our benefit,
> rather than designing in spite of it being true.
>
> However this atomicity guarantee isn't always desirable, as evidenced by
> the original bug report [2]. If you're interacting with a website form for
> example, and a database transaction fails because one of your strings is
> oversize, you don't need to re-input all of your form responses from
> scratch, as there is an application layer/browser in-between to preserve
> the state and retry the transaction.
> And while you could make a convenience/performance/etc argument in that
> situation ("The database should truncate/null-out the oversize string") and
> modern databases often have very expressive DML that would permit such a
> behavior (try-catch, etc), the End-to-End arguments [3] make me believe
> that is a bad design and should be discouraged.
> To that end, I was suggesting ways to push this farther and farther up the
> stack, such as performing record size estimation. This doesn't mean that it
> can't be added at a low level of abstraction, just that we need to make
> sure to exhaust all other alternatives, and justify it with a performance
> benefit.
>
> I was holding off on discussing the literal design until you provided
> concrete performance justification, but to pr

Re: [DISCUSS] KIP-1059: Enable the Producer flush() method to clear the latest send() error

2024-07-19 Thread Greg Harris
Hi Artem and Matthias,

> On the other hand, the effort to prove that
> keeping all records in memory won't break some scenarios (and generally
> breaking one is enough to cause a lot of pain) seems to be significantly
> higher than to prove that setting some flag in some API has pretty much 0
> chance of regression

> in the end, why buffer records twice?

> This way we don't
> ignore the error, we're just changing the method they are delivered.

> Very clean semantics
> which should also address the concern of "non-atomic tx"

I feel like my concerns are being minimized instead of being addressed
in this discussion, and if that's because I'm not expressing them clearly,
I apologize.

Many users come to Kafka with prior expectations, especially when we use
industry-standard terminology like 'Exactly Once Semantics",
"Transactions", "Commit", "Abort". Of course Kafka isn't an ACID-compliant
database, but users will evaluate, discuss, and develop applications with
Kafka through the lens of the ACID principles, because that is the
framework most commonly applied to transactional semantics.
The original design of KIP-98 [1] explicitly mentions atomic commits (with
the same meaning as the A in ACID) as the primary abstraction being added
(reproduced here):

> At the core, transactional guarantees enable applications to produce to
multiple TopicPartitions atomically, ie. all writes to these
TopicPartitions will succeed or fail as a unit.
> Further, since consumer progress is recorded as a write to the offsets
topic, the above capability is leveraged to enable applications to batch
consumed and produced messages into a single atomic unit, ie. a set of
messages may be considered consumed only if the entire
‘consume-transform-produce’ executed in its entirety.

I think it's important to say that to a user, "writes" really means "send()
and commitOffsets() calls", not literal produce requests to Kafka brokers,
and "consume-transform-produce" really means "poll(), transform, send()".
This is because to a user, the implementation within poll() and send() and
the broker are none of their concern, and are intended to be within the
abstraction.
When I say that this feature is a non-atomic commit, I mean that this
feature does not fit the above description, and breaks the transaction
abstraction in a meaningful way. No longer do all writes succeed or fail as
a unit, some failures are permitted to drop data. No longer must a
consume-transform-produce cycle be executed in its entirety, some parts may
be left incomplete.
This means that this method will be difficult to define ("which exceptions
are covered?"), difficult to document ("how do we explain
'not-really-atomic commits' clearly and unambiguously to a potential
user?"), and difficult to compose ("if someone turns this option on, how
does that affect delivery guarantees and opportunities for bugs in
upper layers?").
Users currently rely heavily on analogies to other database systems to make
sense of Kafka's transactions, and we need to use that to our benefit,
rather than designing in spite of it being true.

However this atomicity guarantee isn't always desirable, as evidenced by
the original bug report [2]. If you're interacting with a website form for
example, and a database transaction fails because one of your strings is
oversize, you don't need to re-input all of your form responses from
scratch, as there is an application layer/browser in-between to preserve
the state and retry the transaction.
And while you could make a convenience/performance/etc argument in that
situation ("The database should truncate/null-out the oversize string") and
modern databases often have very expressive DML that would permit such a
behavior (try-catch, etc), the End-to-End arguments [3] make me believe
that is a bad design and should be discouraged.
To that end, I was suggesting ways to push this farther and farther up the
stack, such as performing record size estimation. This doesn't mean that it
can't be added at a low level of abstraction, just that we need to make
sure to exhaust all other alternatives, and justify it with a performance
benefit.

I was holding off on discussing the literal design until you provided
concrete performance justification, but to progress the discussion while
i'm waiting for that, I can give my thoughts:

I don't think an overloaded send() method is appropriate given that this
appears to be a niche use-case, and the send() method itself is probably
the single most important method in the Clients library. The KIP-98 design
was a much more substantial change to the Producer than this KIP, and it
found a way to preserve the original type signature (but added an
exception).
Users picking up the Producer for the first time may see this additional
method, and may spend time trying to understand whether it is something
suitable for their use-case. In the best case, they ignore it and use the
other two signatures. But it is also possible that they will use 

Re: [DISCUSS] KIP-1059: Enable the Producer flush() method to clear the latest send() error

2024-07-19 Thread Matthias J. Sax

For catching client side errors this would work IMHO. I am ok with this.

We throw before we add the record to the batch. Very clean semantics 
which should also address the concern of "non-atomic tx"... The 
exception clearly indicates that the record was not added to the TX, and 
users can react to it accordingly.


We did discuss this idea previously, but did not have a good proposal to 
make it backward compatible. The newly proposed overload would address 
this issue of backward compatibility.


Of course, it might not make it easily extensible in the future for 
broker side errors, but it's unclear anyway right now, if we would even 
get to a solution for broker side errors or not -- so maybe it's ok to 
accept this and drop/ignore the broker side error question for now.




A small follow up thought/question: instead of using a boolean, would we 
actually want to make it a var-arg enum to allow users to enable this 
for certain errors explicitly and individually? Beside the added 
flexibility and fine grain control, a var-arg enum would also make the 
API nicer/cleaner IMHO compare to a boolean.


For convenience, this enum could have an additional `ALL` option (and we 
would call out that if `ALL` is used, new error types might be added in 
future release making the code less safe/robust -- ie, use at your own 
risk only)


This way, we also explicitly document what exception might be thrown in 
the KIP, as we would add an enum for each error type explicitly, and 
also make if future proof for new error types we want to cover -- each 
addition would require a KIP to extend the enum.




-Matthias


On 7/18/24 10:33 PM, Artem Livshits wrote:

Hey folks,

Hopefully not to make this KIP go for another spin :-), but I thought of a
modification that might actually address safety concerns over using flags
to ignore a vaguely specified class of errors.

What if we had the following overload of .send method:

   void send(ProducerRecord record, Callback callback, boolean
throwImmediately)

and if throwImmediately=false, then we behave the same way as now (return
errors via Future and poison transaction) and if throwImmediately=true then
we just throw errors immediately from the send function.  This way we don't
ignore the error, we're just changing the method they are delivered.  Then
KStreams can catch the error for send(record, callback, true) and do
whatever it needs to do.

-Artem


On Mon, Jul 15, 2024 at 4:30 PM Greg Harris 
wrote:


Matthias,

Thank you for rejecting my suggested alternatives. Your responses are the
sorts of things I expected to see summarized in the text of the KIP.

I agree with most of your rejections, except this one:


"Estimation" is not sufficient, but we would need to know it exactly.
And that's an impl detail, given that the message format could change
and we could add new internal fields increasing the message size.


An estimate is certainly going to have an error. But an estimate shouldn't
be treated as exact anyway, there should be an error bound, or "safety
factor" used when interpreting it. For example, if the broker side limit is
1MB, and an estimate could be wrong by ~10%, then computing an estimate and
dropping records >900kb should be sufficient to prevent RTLEs.
This is the sort of estimation that I would expect application developers
could implement, without knowing the exact serialization and protocol
overhead. This could prevent user-originated oversize records from making
it to the producer.

Thanks,
Greg


On Mon, Jul 15, 2024 at 4:08 PM Matthias J. Sax  wrote:


I agree with Alieh and Artem -- in the end, why buffer records twice? We
effectively want to allow to push some error handling (which I btw
consider "business logic") into the producer. IMHO, there is nothing
wrong with it. Dropping a poison pill record is no really a violation of
atomicity from my POV, but a business logic decision to not include a
record in a transaction -- the proposed API just makes it much simpler
to achieve this business logic goal.



For memory size estimation, throughput or message size is actually not
relevant, right? We would need to look at producer buffer size, ie,
`batch.size`, `max.in.flight.request.per.connection` and guesstimate the
number of connections there might be? At least for KS, we don't need to
buffer everything until commit, but only until we get a successful "ack"
back.

Note that KS application not only need to write to (a single) user
result topic, but multiple output topics, as well as repartition and
changelog topics, across all tasks assigned to a thread (ie, producer),
which can easily be 10 tasks or more. If we assume topics with 30
partitions (topics with 50 or more partitions are not uncommon either),
and a producer who must write to 10 different topics, the number of
required connections is very quickly very high, and thus the required
"application buffer space" would be significant.



Your others ideas seems not to be viable alternatives:


S

Re: [DISCUSS] KIP-1059: Enable the Producer flush() method to clear the latest send() error

2024-07-18 Thread Artem Livshits
Hey folks,

Hopefully not to make this KIP go for another spin :-), but I thought of a
modification that might actually address safety concerns over using flags
to ignore a vaguely specified class of errors.

What if we had the following overload of .send method:

  void send(ProducerRecord record, Callback callback, boolean
throwImmediately)

and if throwImmediately=false, then we behave the same way as now (return
errors via Future and poison transaction) and if throwImmediately=true then
we just throw errors immediately from the send function.  This way we don't
ignore the error, we're just changing the method they are delivered.  Then
KStreams can catch the error for send(record, callback, true) and do
whatever it needs to do.

-Artem


On Mon, Jul 15, 2024 at 4:30 PM Greg Harris 
wrote:

> Matthias,
>
> Thank you for rejecting my suggested alternatives. Your responses are the
> sorts of things I expected to see summarized in the text of the KIP.
>
> I agree with most of your rejections, except this one:
>
> > "Estimation" is not sufficient, but we would need to know it exactly.
> > And that's an impl detail, given that the message format could change
> > and we could add new internal fields increasing the message size.
>
> An estimate is certainly going to have an error. But an estimate shouldn't
> be treated as exact anyway, there should be an error bound, or "safety
> factor" used when interpreting it. For example, if the broker side limit is
> 1MB, and an estimate could be wrong by ~10%, then computing an estimate and
> dropping records >900kb should be sufficient to prevent RTLEs.
> This is the sort of estimation that I would expect application developers
> could implement, without knowing the exact serialization and protocol
> overhead. This could prevent user-originated oversize records from making
> it to the producer.
>
> Thanks,
> Greg
>
>
> On Mon, Jul 15, 2024 at 4:08 PM Matthias J. Sax  wrote:
>
> > I agree with Alieh and Artem -- in the end, why buffer records twice? We
> > effectively want to allow to push some error handling (which I btw
> > consider "business logic") into the producer. IMHO, there is nothing
> > wrong with it. Dropping a poison pill record is no really a violation of
> > atomicity from my POV, but a business logic decision to not include a
> > record in a transaction -- the proposed API just makes it much simpler
> > to achieve this business logic goal.
> >
> >
> >
> > For memory size estimation, throughput or message size is actually not
> > relevant, right? We would need to look at producer buffer size, ie,
> > `batch.size`, `max.in.flight.request.per.connection` and guesstimate the
> > number of connections there might be? At least for KS, we don't need to
> > buffer everything until commit, but only until we get a successful "ack"
> > back.
> >
> > Note that KS application not only need to write to (a single) user
> > result topic, but multiple output topics, as well as repartition and
> > changelog topics, across all tasks assigned to a thread (ie, producer),
> > which can easily be 10 tasks or more. If we assume topics with 30
> > partitions (topics with 50 or more partitions are not uncommon either),
> > and a producer who must write to 10 different topics, the number of
> > required connections is very quickly very high, and thus the required
> > "application buffer space" would be significant.
> >
> >
> >
> > Your others ideas seems not to be viable alternatives:
> >
> > > Streams users that specifically want to drop oversize records can
> > > estimate the size of their data and drop records which are too
> > > large, enforcing their own limits that are lower than the Kafka limits.
> >
> > "Estimation" is not sufficient, but we would need to know it exactly.
> > And that's an impl detail, given that the message format could change
> > and we could add new internal fields increasing the message size. The
> > idea to add some `producer.serializedRecordSize()` helper method was
> > discussed, but it's a very ugly API and clumsy to use -- also, the user
> > code would need to know the producer config which it might not have
> > access to (as it might get passed in from some config file; and it might
> > also be changed).
> >
> > Some other alternative we also discussed was, to let `send()` throw an
> > exception for a "record too large" case directly. However, this solution
> > raises backward compatibly concerns, and it might also not help us to
> > extend the solution in the future (eg, tackle broker side errors). So we
> > discarded this idea.
> >
> >
> >
> > > Streams users that want CONTINUE semantics can use at_least_once
> > > semantics
> >
> > Not really. EOS is mainly about not having duplicates in the result, but
> > at-least-once cannot provide this guarantee. (Even if I repeat my self:
> > but dropping a poison pill record based on a business logic decision is
> > not data loss, but effectively a business logic filter...)
> >
> >
> >
> > > Streams itse

Re: [DISCUSS] KIP-1059: Enable the Producer flush() method to clear the latest send() error

2024-07-15 Thread Greg Harris
Matthias,

Thank you for rejecting my suggested alternatives. Your responses are the
sorts of things I expected to see summarized in the text of the KIP.

I agree with most of your rejections, except this one:

> "Estimation" is not sufficient, but we would need to know it exactly.
> And that's an impl detail, given that the message format could change
> and we could add new internal fields increasing the message size.

An estimate is certainly going to have an error. But an estimate shouldn't
be treated as exact anyway, there should be an error bound, or "safety
factor" used when interpreting it. For example, if the broker side limit is
1MB, and an estimate could be wrong by ~10%, then computing an estimate and
dropping records >900kb should be sufficient to prevent RTLEs.
This is the sort of estimation that I would expect application developers
could implement, without knowing the exact serialization and protocol
overhead. This could prevent user-originated oversize records from making
it to the producer.

Thanks,
Greg


On Mon, Jul 15, 2024 at 4:08 PM Matthias J. Sax  wrote:

> I agree with Alieh and Artem -- in the end, why buffer records twice? We
> effectively want to allow to push some error handling (which I btw
> consider "business logic") into the producer. IMHO, there is nothing
> wrong with it. Dropping a poison pill record is no really a violation of
> atomicity from my POV, but a business logic decision to not include a
> record in a transaction -- the proposed API just makes it much simpler
> to achieve this business logic goal.
>
>
>
> For memory size estimation, throughput or message size is actually not
> relevant, right? We would need to look at producer buffer size, ie,
> `batch.size`, `max.in.flight.request.per.connection` and guesstimate the
> number of connections there might be? At least for KS, we don't need to
> buffer everything until commit, but only until we get a successful "ack"
> back.
>
> Note that KS application not only need to write to (a single) user
> result topic, but multiple output topics, as well as repartition and
> changelog topics, across all tasks assigned to a thread (ie, producer),
> which can easily be 10 tasks or more. If we assume topics with 30
> partitions (topics with 50 or more partitions are not uncommon either),
> and a producer who must write to 10 different topics, the number of
> required connections is very quickly very high, and thus the required
> "application buffer space" would be significant.
>
>
>
> Your others ideas seems not to be viable alternatives:
>
> > Streams users that specifically want to drop oversize records can
> > estimate the size of their data and drop records which are too
> > large, enforcing their own limits that are lower than the Kafka limits.
>
> "Estimation" is not sufficient, but we would need to know it exactly.
> And that's an impl detail, given that the message format could change
> and we could add new internal fields increasing the message size. The
> idea to add some `producer.serializedRecordSize()` helper method was
> discussed, but it's a very ugly API and clumsy to use -- also, the user
> code would need to know the producer config which it might not have
> access to (as it might get passed in from some config file; and it might
> also be changed).
>
> Some other alternative we also discussed was, to let `send()` throw an
> exception for a "record too large" case directly. However, this solution
> raises backward compatibly concerns, and it might also not help us to
> extend the solution in the future (eg, tackle broker side errors). So we
> discarded this idea.
>
>
>
> > Streams users that want CONTINUE semantics can use at_least_once
> > semantics
>
> Not really. EOS is mainly about not having duplicates in the result, but
> at-least-once cannot provide this guarantee. (Even if I repeat my self:
> but dropping a poison pill record based on a business logic decision is
> not data loss, but effectively a business logic filter...)
>
>
>
> > Streams itself can store record hashes/coordinates and fast rewind to
> > the end of the last transaction, recomputing data rather than storing it.
>
> Given the very complex nature of topologies, with joins, aggregations,
> flatmaps etc, this is a 100x more complex solution and not viable in
> practice.
>
>
>
> > Streams can define exactly_once + CONTINUE semantics to permit the whole
> > transaction to be dropped, because it would allow the next batch to be
> > started processing.
>
> Would this not be much worse? I have a single poison pill record and
> would need to drop a full tx (this could be tens of thousands of
> records...). Also, given that KS write into changelog topic in the same
> TX, this could break the whole application.
>
>
>
> > Streams can emit records with both a transactional and non-transactional
> > producer if some records are not critical-path
>
> We (1) already have a "too many connections" problem with KS so using
> move clients is something we

Re: [DISCUSS] KIP-1059: Enable the Producer flush() method to clear the latest send() error

2024-07-15 Thread Matthias J. Sax
I agree with Alieh and Artem -- in the end, why buffer records twice? We 
effectively want to allow to push some error handling (which I btw 
consider "business logic") into the producer. IMHO, there is nothing 
wrong with it. Dropping a poison pill record is no really a violation of 
atomicity from my POV, but a business logic decision to not include a 
record in a transaction -- the proposed API just makes it much simpler 
to achieve this business logic goal.




For memory size estimation, throughput or message size is actually not 
relevant, right? We would need to look at producer buffer size, ie, 
`batch.size`, `max.in.flight.request.per.connection` and guesstimate the 
number of connections there might be? At least for KS, we don't need to 
buffer everything until commit, but only until we get a successful "ack" 
back.


Note that KS application not only need to write to (a single) user 
result topic, but multiple output topics, as well as repartition and 
changelog topics, across all tasks assigned to a thread (ie, producer), 
which can easily be 10 tasks or more. If we assume topics with 30 
partitions (topics with 50 or more partitions are not uncommon either), 
and a producer who must write to 10 different topics, the number of 
required connections is very quickly very high, and thus the required 
"application buffer space" would be significant.




Your others ideas seems not to be viable alternatives:


Streams users that specifically want to drop oversize records can
estimate the size of their data and drop records which are too
large, enforcing their own limits that are lower than the Kafka limits.


"Estimation" is not sufficient, but we would need to know it exactly. 
And that's an impl detail, given that the message format could change 
and we could add new internal fields increasing the message size. The 
idea to add some `producer.serializedRecordSize()` helper method was 
discussed, but it's a very ugly API and clumsy to use -- also, the user 
code would need to know the producer config which it might not have 
access to (as it might get passed in from some config file; and it might 
also be changed).


Some other alternative we also discussed was, to let `send()` throw an 
exception for a "record too large" case directly. However, this solution 
raises backward compatibly concerns, and it might also not help us to 
extend the solution in the future (eg, tackle broker side errors). So we 
discarded this idea.





Streams users that want CONTINUE semantics can use at_least_once
semantics


Not really. EOS is mainly about not having duplicates in the result, but 
at-least-once cannot provide this guarantee. (Even if I repeat my self: 
but dropping a poison pill record based on a business logic decision is 
not data loss, but effectively a business logic filter...)





Streams itself can store record hashes/coordinates and fast rewind to
the end of the last transaction, recomputing data rather than storing it.


Given the very complex nature of topologies, with joins, aggregations, 
flatmaps etc, this is a 100x more complex solution and not viable in 
practice.





Streams can define exactly_once + CONTINUE semantics to permit the whole
transaction to be dropped, because it would allow the next batch to be
started processing.


Would this not be much worse? I have a single poison pill record and 
would need to drop a full tx (this could be tens of thousands of 
records...). Also, given that KS write into changelog topic in the same 
TX, this could break the whole application.





Streams can emit records with both a transactional and non-transactional
producer if some records are not critical-path


We (1) already have a "too many connections" problem with KS so using 
move clients is something we try to avoid (and we actually hope to 
reduce the number of client and connection mid to long term), (2) this 
would be very hard to express at the API level to the user, and (3) it 
would provide very weird semantics.





they should optimize for smaller transactions,


IMHO, this would not work in practice because transaction have a high 
overhead and commit-interval is used to tradeoff throughput vs 
end-to-end latency. Given certain throughput requirement, it would not 
be possible to just use a lower commit interval to reduce memory 
requirements.




-Matthias




On 7/15/24 2:25 PM, Artem Livshits wrote:

Hi Greg,


This makes me think that this IGNORE_SEND_ERRORS covers an arbitrary set

of error conditions that may be expanded in the future, possibly to cover
the broker side RecordTooLargeException.

I don't think it contradicts what I said (the keyword here is "in the
future") -- with the current functionality, the correct way to handle RTLE
is by only letting the client ignore client-originated RTLE (this can be
easily implemented on the client side).  In the future, we can improve on
that by making the broker return a different exception for batch-too-large
case, then the produ

Re: [DISCUSS] KIP-1059: Enable the Producer flush() method to clear the latest send() error

2024-07-15 Thread Artem Livshits
Hi Greg,

> This makes me think that this IGNORE_SEND_ERRORS covers an arbitrary set
of error conditions that may be expanded in the future, possibly to cover
the broker side RecordTooLargeException.

I don't think it contradicts what I said (the keyword here is "in the
future") -- with the current functionality, the correct way to handle RTLE
is by only letting the client ignore client-originated RTLE (this can be
easily implemented on the client side).  In the future, we can improve on
that by making the broker return a different exception for batch-too-large
case, then the producer would be able to return broker side exceptions as
well (and if the application chooses to ignore it -- it will be able to,
but it would be an explicit choice rather than ignoring it by mistake), in
this case the producer client would encapsulate backward compatibility
logic when it connects to older brokers to make sure the the application
doesn't accidentally gets RTLE originated by the old broker.  This
functionality is obviously more involved and we'll need to see if going all
the way is justified, but the partial client-only solution doesn't close
the door.

So one way to look at the current situation is the following:

1. We can do a low effort partial solution to solve a real existing
problem.  We can easily prove that it would do exactly what it needs to do
with minimal risk of regression.
2. We have a path to a more comprehensive solution, so if we justify the
effort required for that, we can get there.

BTW, as a side note (I think a saw a question in the thread), we do try to
introduce error categories here
https://cwiki.apache.org/confluence/display/KAFKA/KIP-1050%3A+Consistent+error+handling+for+Transactions
so eventually we may have a better classification for the errors.

> "if a streams producer is producing 1MB/s, and the commit interval is 1
hour, I expect 3600MB of additional heap needed ...

Agree, that would be ideal.  On the other hand, the effort to prove that
keeping all records in memory won't break some scenarios (and generally
breaking one is enough to cause a lot of pain) seems to be significantly
higher than to prove that setting some flag in some API has pretty much 0
chance of regression (we basically have a flag to say "unfix KAFKA-9279" so
we're getting to fairly "known good" state).  I'll let KStream folks
comment on this one (and we still need to solve the problem of accidental
handling of RTLE originated from broker, so some KIP would be required to
somehow help to differentiate those).

-Artem

On Mon, Jul 15, 2024 at 1:31 PM Greg Harris 
wrote:

> Hi Artem,
>
> Thank you for clarifying as I'm joining the conversation late and may have
> some misconceptions.
>
> > Because of this, a more "complete" solution that
> > allows ignoring RecordTooLargeException regardless of its origin is
> > actually incorrect, while a "partial" solution that allows ignoring
> > RecordTooLargeException only originating in client code accomplishes the
> > required functionality.
>
> This is not how I understood this feature. Above Matthias said the
> following:
>
> > We can do
> > follow up KIP for other errors on an on-demand basis and fix-forward /
> > enlarge the scope successively.
>
> This makes me think that this IGNORE_SEND_ERRORS covers an arbitrary set of
> error conditions that may be expanded in the future, possibly to cover the
> broker side RecordTooLargeException.
>
> > Obviously, we could solve this problem by changing logic in the
> > broker to return a different error when the batch is too large, but right
> > now this is not the case
>
> If the broker/wire protocol isn't ready for these errors to be propagated,
> then I don't think we're ready to add this API. It's going to be
> under-generalized, and there's a decent chance that we're going to regret
> the design choices in the future. And users that expect it to be fully
> generalized are going to be disappointed when they don't read the fine
> print and still get faulted by non-covered errors.
>
> > AL2.  In a high performance system, "just an optimization" can be a
> > functional requirement ...
> >  I just wanted to make the point that we shouldn't necessarily dismiss
> > API changes that allow for optimizations.
>
> My earlier statement didn't dismiss this feature as "just an optimization",
> actually the opposite. I said that performance could be a justification,
> but only if it is quantified and stated explicitly. We shouldn't be voting
> on hand-wavy optimizations, we should be voting on things that are
> quantifiable.
> For example an analysis like the following would facilitate further
> discussion: "if a streams producer is producing 1MB/s, and the commit
> interval is 1 hour, I expect 3600MB of additional heap needed per
> producer". We can then discuss whether we expect higher or lower
> throughput, commit intervals, or heap usage to determine what the operating
> envelope of this feature could be.
> If there are a substantial num

Re: [DISCUSS] KIP-1059: Enable the Producer flush() method to clear the latest send() error

2024-07-15 Thread Greg Harris
Hi Artem,

Thank you for clarifying as I'm joining the conversation late and may have
some misconceptions.

> Because of this, a more "complete" solution that
> allows ignoring RecordTooLargeException regardless of its origin is
> actually incorrect, while a "partial" solution that allows ignoring
> RecordTooLargeException only originating in client code accomplishes the
> required functionality.

This is not how I understood this feature. Above Matthias said the
following:

> We can do
> follow up KIP for other errors on an on-demand basis and fix-forward /
> enlarge the scope successively.

This makes me think that this IGNORE_SEND_ERRORS covers an arbitrary set of
error conditions that may be expanded in the future, possibly to cover the
broker side RecordTooLargeException.

> Obviously, we could solve this problem by changing logic in the
> broker to return a different error when the batch is too large, but right
> now this is not the case

If the broker/wire protocol isn't ready for these errors to be propagated,
then I don't think we're ready to add this API. It's going to be
under-generalized, and there's a decent chance that we're going to regret
the design choices in the future. And users that expect it to be fully
generalized are going to be disappointed when they don't read the fine
print and still get faulted by non-covered errors.

> AL2.  In a high performance system, "just an optimization" can be a
> functional requirement ...
>  I just wanted to make the point that we shouldn't necessarily dismiss
> API changes that allow for optimizations.

My earlier statement didn't dismiss this feature as "just an optimization",
actually the opposite. I said that performance could be a justification,
but only if it is quantified and stated explicitly. We shouldn't be voting
on hand-wavy optimizations, we should be voting on things that are
quantifiable.
For example an analysis like the following would facilitate further
discussion: "if a streams producer is producing 1MB/s, and the commit
interval is 1 hour, I expect 3600MB of additional heap needed per
producer". We can then discuss whether we expect higher or lower
throughput, commit intervals, or heap usage to determine what the operating
envelope of this feature could be.
If there are a substantial number of users that have high throughput, long
commit intervals, _and_ RTLEs, then this feature could make sense. If not,
then the downsides of this feature (complication of the API,
under-specification of the error coverage, etc) look unjustified. In fact,
if the number of users regularly encountering RTLEs is sufficiently small,
I would strongly advocate for an application-specific workaround instead of
trying to fix this in Streams, or make memory buffering an optional feature
in streams.

Thanks,
Greg

On Mon, Jul 15, 2024 at 1:29 PM Greg Harris  wrote:

> Hi Alieh,
>
> Thanks for your response.
>
> > what does a user do
> > after a transaction is failed due to a `too-large-record `exception? They
> > will submit the same batch without the problematic record again.
>
> If they re-submit the same record, they are indicating that this record is
> an integral part of the transaction, and the transaction should only be
> committed with it present. If the record isn't integral to the transaction,
> they shouldn't submit it as part of the transaction.
>
> > Regarding your solution to solve the issue application-side:  I am a
> > bit hesitant to keep all sent records in memory since I think buffering
> > records twice (both in Streams and Producer) would not be an efficient
> > solution.
>
> I understand your hesitation, and this touches on the "performance" caveat
> of the end-to-end arguments in system design. There are no perfect designs,
> and some API cleanliness may be sacrificed in favor of more performant
> solutions. You would need to make a concrete and convincing argument that
> the performance of this solution would be better than every alternative. To
> that end, I would recommend that you add more to the "Rejected
> Alternatives" section, as that is going to carry this proposal.
> Some alternatives that I can think of, but which aren't necessarily better:
> 1. Streams users that specifically want to drop oversize records can
> estimate the size of their data and drop records which are too
> large, enforcing their own limits that are lower than the Kafka limits.
> 2. Streams users that want CONTINUE semantics can use at_least_once
> semantics
> 3. Streams itself can store record hashes/coordinates and fast rewind to
> the end of the last transaction, recomputing data rather than storing it.
> 4. Streams can define exactly_once + CONTINUE semantics to permit the
> whole transaction to be dropped, because it would allow the next batch to
> be started processing.
> 5. Streams can emit records with both a transactional and
> non-transactional producer if some records are not critical-path
>
> To generalize this point: Suppose an application tries t

Re: [DISCUSS] KIP-1059: Enable the Producer flush() method to clear the latest send() error

2024-07-15 Thread Greg Harris
Hi Alieh,

Thanks for your response.

> what does a user do
> after a transaction is failed due to a `too-large-record `exception? They
> will submit the same batch without the problematic record again.

If they re-submit the same record, they are indicating that this record is
an integral part of the transaction, and the transaction should only be
committed with it present. If the record isn't integral to the transaction,
they shouldn't submit it as part of the transaction.

> Regarding your solution to solve the issue application-side:  I am a
> bit hesitant to keep all sent records in memory since I think buffering
> records twice (both in Streams and Producer) would not be an efficient
> solution.

I understand your hesitation, and this touches on the "performance" caveat
of the end-to-end arguments in system design. There are no perfect designs,
and some API cleanliness may be sacrificed in favor of more performant
solutions. You would need to make a concrete and convincing argument that
the performance of this solution would be better than every alternative. To
that end, I would recommend that you add more to the "Rejected
Alternatives" section, as that is going to carry this proposal.
Some alternatives that I can think of, but which aren't necessarily better:
1. Streams users that specifically want to drop oversize records can
estimate the size of their data and drop records which are too
large, enforcing their own limits that are lower than the Kafka limits.
2. Streams users that want CONTINUE semantics can use at_least_once
semantics
3. Streams itself can store record hashes/coordinates and fast rewind to
the end of the last transaction, recomputing data rather than storing it.
4. Streams can define exactly_once + CONTINUE semantics to permit the whole
transaction to be dropped, because it would allow the next batch to be
started processing.
5. Streams can emit records with both a transactional and non-transactional
producer if some records are not critical-path

To generalize this point: Suppose an application tries to minimize storage
costs by having only one party responsible for a piece of data at a time.
They initially have the data, call send(), and want to know the earliest
time they can forget the data and transfer the responsibility to Kafka.
With a non-transactional producer, they are responsible for the data until
the send() callback has succeeded. With a transactional producer, they are
responsible for the data until commitTransaction() has succeeded.
With this proposed change that makes the producer tolerate
too-large-exceptions, applications are still responsible for storing their
data until commitTransaction() has succeeded, because abortTransaction()
could have also been called, or the producer could have been fenced, or any
number of other failures could have occurred. This feature does not enable
Streams to drop responsibility earlier, it carves out a specific situation
in which it doesn't have to rewind processing, which is a performance
concern.

For Streams and the general case, if an application is trying to optimize
storage costs, they should optimize for smaller transactions, because this
both lowers the bound on record re-delivery and lowers the likelihood of a
bad record being included in any individual transaction.

Thanks,
Greg

On Mon, Jul 15, 2024 at 12:35 PM Artem Livshits
 wrote:

> Hi Greg,
>
> What you say makes a lot of sense.  I just wanted to clarify a couple of
> subtle points.
>
> AL1. There is a functional reason to handle errors that happen on send
> (oginate in the producer logic in the client) vs. errors that are returned
> from the broker.  The problem is that RecordTooLargeException is returned
> in two cases: (1) the producer logic on the client checks that record is
> too large and throws the exception before doing anything with this -- this
> is very "clean" situation with one specific record being marked as "poison
> pill" and rejected; (2) the broker throws the same error if the batch is
> too large -- the batch may include multiple records and none of them would
> necessarily be a "poison pill" record, it's just a random misconfiguration
> of client vs. broker.  Because of this, a more "complete" solution that
> allows ignoring RecordTooLargeException regardless of its origin is
> actually incorrect, while a "partial" solution that allows ignoring
> RecordTooLargeException only originating in client code accomplishes the
> required functionality.  This is an important nuance and should be added to
> the KIP.  Obviously, we could solve this problem by changing logic in the
> broker to return a different error when the batch is too large, but right
> now this is not the case (and to have the correct error handling we'd need
> to know the version of the broker so we can only drop the records if the
> error is returned from a broker that knows to return a different error).
>
> AL2.  In a high performance system, "just an optimization" can be a
> funct

Re: [DISCUSS] KIP-1059: Enable the Producer flush() method to clear the latest send() error

2024-07-15 Thread Artem Livshits
Hi Greg,

What you say makes a lot of sense.  I just wanted to clarify a couple of
subtle points.

AL1. There is a functional reason to handle errors that happen on send
(oginate in the producer logic in the client) vs. errors that are returned
from the broker.  The problem is that RecordTooLargeException is returned
in two cases: (1) the producer logic on the client checks that record is
too large and throws the exception before doing anything with this -- this
is very "clean" situation with one specific record being marked as "poison
pill" and rejected; (2) the broker throws the same error if the batch is
too large -- the batch may include multiple records and none of them would
necessarily be a "poison pill" record, it's just a random misconfiguration
of client vs. broker.  Because of this, a more "complete" solution that
allows ignoring RecordTooLargeException regardless of its origin is
actually incorrect, while a "partial" solution that allows ignoring
RecordTooLargeException only originating in client code accomplishes the
required functionality.  This is an important nuance and should be added to
the KIP.  Obviously, we could solve this problem by changing logic in the
broker to return a different error when the batch is too large, but right
now this is not the case (and to have the correct error handling we'd need
to know the version of the broker so we can only drop the records if the
error is returned from a broker that knows to return a different error).

AL2.  In a high performance system, "just an optimization" can be a
functional requirement -- if a solution impacts memory or computational
complexity (in the sense of bigO notation) on the main code path I can
justify changing APIs to avoid such an impact.  I'll let KStream folks
comment on whether an implementation that requires storing records in
memory actually violates the computational complexity on the main code
path, I just wanted to make the point that we shouldn't necessarily dismiss
API changes that allow for optimizations.

-Artem

On Fri, Jul 12, 2024 at 1:07 PM Greg Harris 
wrote:

> Hi all,
>
> Alieh, thanks for the KIP! And everyone else, thanks for the robust
> discussion.
>
> I understand that there are situations in which users desire that the
> pipeline "just keep working" and skip errors. However, I question whether
> it is appropriate to support/encourage this behavior via inclusion in the
> Producer API.
> This feature is essentially a "non-atomic transaction", as it allows
> commits in which not all records passed to send() ultimately get committed.
> As atomicity is one of the most important semantics associated with
> transactions, I question whether there are users other than Streams that
> would choose non-atomic transactions over a traditional/idempotent
> producer.
> Some cursory research shows that non-atomic transactions may be present in
> other databases, but is actively discouraged due to the complexity they add
> to error-handling. [1]
>
> I'd like to invoke the End-to-End Arguments in System Design [2] here, and
> recommend that this behavior may be present in Streams, but should not be
> in the Producer.
> 1. Dropping records that cause errors is already expressible via the
> current Producer API. You can store the records in-memory after calling
> send(), wait for a successful no-error flush() before calling
> commitTransaction() and allowing the record to be garbage collected. If
> errors occur, abortTransaction() and re-submit the records.
> 2. Implementing this inside the Producer API is complex and difficult to
> holistically define in a way that we won't regret or need to change later.
> I think some of the disagreement in this thread originates from this, and I
> don't find the proposed API satisfactory.
> 3. The performance improvement of including this change in the lower level
> needs to be quantified in order to be a justification, and I don't see any
> analysis about this.
>
> I imagine that the alternative implementation I suggested in (1) would also
> enable more expressive error handlers in Streams, if such a thing was
> desired. Keeping the record around until after the transaction is committed
> would enable a DLQ or passing the erroneous record to the error handler.
>
> I think that the current pattern of the application being responsible for
> providing good data to the producer is very reasonable; Having the producer
> responsible for implementing the application's error handling of bad data
> is not something I can support.
>
> Thanks,
> Greg
>
> [1] https://www.sommarskog.se/error_handling/Part1.html
> [2] https://web.mit.edu/Saltzer/www/publications/endtoend/endtoend.pdf
>
> On Fri, Jul 12, 2024 at 8:52 AM Justine Olshan
> 
> wrote:
>
> > Can we update the KIP to clearly document these decisions?
> >
> > Thanks,
> >
> > Justine
> >
> > On Tue, Jul 9, 2024 at 9:25 AM Andrew Schofield <
> andrew_schofi...@live.com
> > >
> > wrote:
> >
> > > Hi Chris,
> > > As it stands, the error handl

Re: [DISCUSS] KIP-1059: Enable the Producer flush() method to clear the latest send() error

2024-07-15 Thread Alieh Saeedi
Hey Greg,

thanks for the feedback.

I can understand your concern about atomicity, but what does a user do
after a transaction is failed due to a `too-large-record `exception? They
will submit the same batch without the problematic record again. What we
are providing is actually a better/more convenient way of doing that.

Regarding your solution to solve the issue application-side:  I am a
bit hesitant to keep all sent records in memory since I think buffering
records twice (both in Streams and Producer) would not be an efficient
solution.

Cheers,
Alieh

On Fri, Jul 12, 2024 at 10:07 PM Greg Harris 
wrote:

> Hi all,
>
> Alieh, thanks for the KIP! And everyone else, thanks for the robust
> discussion.
>
> I understand that there are situations in which users desire that the
> pipeline "just keep working" and skip errors. However, I question whether
> it is appropriate to support/encourage this behavior via inclusion in the
> Producer API.
> This feature is essentially a "non-atomic transaction", as it allows
> commits in which not all records passed to send() ultimately get committed.
> As atomicity is one of the most important semantics associated with
> transactions, I question whether there are users other than Streams that
> would choose non-atomic transactions over a traditional/idempotent
> producer.
> Some cursory research shows that non-atomic transactions may be present in
> other databases, but is actively discouraged due to the complexity they add
> to error-handling. [1]
>
> I'd like to invoke the End-to-End Arguments in System Design [2] here, and
> recommend that this behavior may be present in Streams, but should not be
> in the Producer.
> 1. Dropping records that cause errors is already expressible via the
> current Producer API. You can store the records in-memory after calling
> send(), wait for a successful no-error flush() before calling
> commitTransaction() and allowing the record to be garbage collected. If
> errors occur, abortTransaction() and re-submit the records.
> 2. Implementing this inside the Producer API is complex and difficult to
> holistically define in a way that we won't regret or need to change later.
> I think some of the disagreement in this thread originates from this, and I
> don't find the proposed API satisfactory.
> 3. The performance improvement of including this change in the lower level
> needs to be quantified in order to be a justification, and I don't see any
> analysis about this.
>
> I imagine that the alternative implementation I suggested in (1) would also
> enable more expressive error handlers in Streams, if such a thing was
> desired. Keeping the record around until after the transaction is committed
> would enable a DLQ or passing the erroneous record to the error handler.
>
> I think that the current pattern of the application being responsible for
> providing good data to the producer is very reasonable; Having the producer
> responsible for implementing the application's error handling of bad data
> is not something I can support.
>
> Thanks,
> Greg
>
> [1] https://www.sommarskog.se/error_handling/Part1.html
> [2] https://web.mit.edu/Saltzer/www/publications/endtoend/endtoend.pdf
>
> On Fri, Jul 12, 2024 at 8:52 AM Justine Olshan
> 
> wrote:
>
> > Can we update the KIP to clearly document these decisions?
> >
> > Thanks,
> >
> > Justine
> >
> > On Tue, Jul 9, 2024 at 9:25 AM Andrew Schofield <
> andrew_schofi...@live.com
> > >
> > wrote:
> >
> > > Hi Chris,
> > > As it stands, the error handling for transactions in KafkaProducer is
> not
> > > ideal. There’s no reason why a failed operation should fail a
> transaction
> > > provided that the application can tell that the operation was not
> > included
> > > in the transaction and then make its own decision whether to continue
> or
> > > back out. So, I think I disagree with the original premise of a
> > client-side
> > > error state for a transaction, but we are where we are.
> > >
> > > When I voted, I did not expect the KIP to handle ALL errors which could
> > > conceivably be handled. I did expect it to handle client-side send
> errors
> > > that would cause a record to be rejected from a batch before sending
> to a
> > > broker. I think that it does make the KafkaProducer interface very
> > slightly
> > > more complicated, but the new option is a clear improvement and I
> > > don’t see anyone getting into a mess using it.
> > >
> > > I think broker-side errors are more tricky and I don’t think an
> overload
> > > on the send() method is going to do the job. I don’t see that as a
> > problem
> > > with the KIP, just that the underlying RPCs and behaviour is not very
> > > amenable to record-specific error handling. The Produce RPC is a
> > > complicated beast which can include a set of records for mutiple
> > > topic-partitions. Although ProduceResponse v10 does include record
> > > errors, I don’t believe this is surfaced in the client. Let’s imagine
> > > something
> > > like broker-si

Re: [DISCUSS] KIP-1059: Enable the Producer flush() method to clear the latest send() error

2024-07-12 Thread Greg Harris
Hi all,

Alieh, thanks for the KIP! And everyone else, thanks for the robust
discussion.

I understand that there are situations in which users desire that the
pipeline "just keep working" and skip errors. However, I question whether
it is appropriate to support/encourage this behavior via inclusion in the
Producer API.
This feature is essentially a "non-atomic transaction", as it allows
commits in which not all records passed to send() ultimately get committed.
As atomicity is one of the most important semantics associated with
transactions, I question whether there are users other than Streams that
would choose non-atomic transactions over a traditional/idempotent producer.
Some cursory research shows that non-atomic transactions may be present in
other databases, but is actively discouraged due to the complexity they add
to error-handling. [1]

I'd like to invoke the End-to-End Arguments in System Design [2] here, and
recommend that this behavior may be present in Streams, but should not be
in the Producer.
1. Dropping records that cause errors is already expressible via the
current Producer API. You can store the records in-memory after calling
send(), wait for a successful no-error flush() before calling
commitTransaction() and allowing the record to be garbage collected. If
errors occur, abortTransaction() and re-submit the records.
2. Implementing this inside the Producer API is complex and difficult to
holistically define in a way that we won't regret or need to change later.
I think some of the disagreement in this thread originates from this, and I
don't find the proposed API satisfactory.
3. The performance improvement of including this change in the lower level
needs to be quantified in order to be a justification, and I don't see any
analysis about this.

I imagine that the alternative implementation I suggested in (1) would also
enable more expressive error handlers in Streams, if such a thing was
desired. Keeping the record around until after the transaction is committed
would enable a DLQ or passing the erroneous record to the error handler.

I think that the current pattern of the application being responsible for
providing good data to the producer is very reasonable; Having the producer
responsible for implementing the application's error handling of bad data
is not something I can support.

Thanks,
Greg

[1] https://www.sommarskog.se/error_handling/Part1.html
[2] https://web.mit.edu/Saltzer/www/publications/endtoend/endtoend.pdf

On Fri, Jul 12, 2024 at 8:52 AM Justine Olshan 
wrote:

> Can we update the KIP to clearly document these decisions?
>
> Thanks,
>
> Justine
>
> On Tue, Jul 9, 2024 at 9:25 AM Andrew Schofield  >
> wrote:
>
> > Hi Chris,
> > As it stands, the error handling for transactions in KafkaProducer is not
> > ideal. There’s no reason why a failed operation should fail a transaction
> > provided that the application can tell that the operation was not
> included
> > in the transaction and then make its own decision whether to continue or
> > back out. So, I think I disagree with the original premise of a
> client-side
> > error state for a transaction, but we are where we are.
> >
> > When I voted, I did not expect the KIP to handle ALL errors which could
> > conceivably be handled. I did expect it to handle client-side send errors
> > that would cause a record to be rejected from a batch before sending to a
> > broker. I think that it does make the KafkaProducer interface very
> slightly
> > more complicated, but the new option is a clear improvement and I
> > don’t see anyone getting into a mess using it.
> >
> > I think broker-side errors are more tricky and I don’t think an overload
> > on the send() method is going to do the job. I don’t see that as a
> problem
> > with the KIP, just that the underlying RPCs and behaviour is not very
> > amenable to record-specific error handling. The Produce RPC is a
> > complicated beast which can include a set of records for mutiple
> > topic-partitions. Although ProduceResponse v10 does include record
> > errors, I don’t believe this is surfaced in the client. Let’s imagine
> > something
> > like broker-side record validation which barfs on one record. Failing an
> > entire batch is easier, but less useful if the problem is related to one
> > record.
> >
> > In summary, I’m happy that my vote stands, and I am happy with the KIP
> > only supporting client-side record errors.
> >
> > Thanks,
> > Andrew
> >
> > > On 8 Jul 2024, at 16:37, Chris Egerton 
> wrote:
> > >
> > > Hi Alieh,
> > >
> > > Can you clarify why broker-side errors shouldn't be covered? The only
> > real
> > > rationale I can come up with is that it's easier to implement.
> > >
> > > "Things were better for Kafka Streams before KAFKA-9279 was fixed"
> isn't
> > > very convincing, because Kafka Streams is not the only user of the Java
> > > producer client. And for others, especially new users, I doubt that
> this
> > > new API we're proposing would make sense wit

Re: [DISCUSS] KIP-1059: Enable the Producer flush() method to clear the latest send() error

2024-07-12 Thread Justine Olshan
Can we update the KIP to clearly document these decisions?

Thanks,

Justine

On Tue, Jul 9, 2024 at 9:25 AM Andrew Schofield 
wrote:

> Hi Chris,
> As it stands, the error handling for transactions in KafkaProducer is not
> ideal. There’s no reason why a failed operation should fail a transaction
> provided that the application can tell that the operation was not included
> in the transaction and then make its own decision whether to continue or
> back out. So, I think I disagree with the original premise of a client-side
> error state for a transaction, but we are where we are.
>
> When I voted, I did not expect the KIP to handle ALL errors which could
> conceivably be handled. I did expect it to handle client-side send errors
> that would cause a record to be rejected from a batch before sending to a
> broker. I think that it does make the KafkaProducer interface very slightly
> more complicated, but the new option is a clear improvement and I
> don’t see anyone getting into a mess using it.
>
> I think broker-side errors are more tricky and I don’t think an overload
> on the send() method is going to do the job. I don’t see that as a problem
> with the KIP, just that the underlying RPCs and behaviour is not very
> amenable to record-specific error handling. The Produce RPC is a
> complicated beast which can include a set of records for mutiple
> topic-partitions. Although ProduceResponse v10 does include record
> errors, I don’t believe this is surfaced in the client. Let’s imagine
> something
> like broker-side record validation which barfs on one record. Failing an
> entire batch is easier, but less useful if the problem is related to one
> record.
>
> In summary, I’m happy that my vote stands, and I am happy with the KIP
> only supporting client-side record errors.
>
> Thanks,
> Andrew
>
> > On 8 Jul 2024, at 16:37, Chris Egerton  wrote:
> >
> > Hi Alieh,
> >
> > Can you clarify why broker-side errors shouldn't be covered? The only
> real
> > rationale I can come up with is that it's easier to implement.
> >
> > "Things were better for Kafka Streams before KAFKA-9279 was fixed" isn't
> > very convincing, because Kafka Streams is not the only user of the Java
> > producer client. And for others, especially new users, I doubt that this
> > new API we're proposing would make sense without having to consult a lot
> of
> > historical context.
> >
> > I also don't think that most users will know or even care about the
> > distinction between errors that cause a record to fail before it's added
> to
> > a batch vs. after. If you were writing a producer application of your
> own,
> > and you wanted to handle RecordTooLargeException instances by dropping a
> > record without aborting a transaction, would you care about whether it
> was
> > your client or your broker that balked? Would you be happy if you wrote
> > logic expecting that that problem was solved once and for all, only to
> > learn that it could still affect you in other circumstances? Or,
> > alternatively, would you be happy if you wanted to solve that problem and
> > found an API that seemed to do exactly what you wanted, but after reading
> > the fine print, realized you'd have to do it yourself instead?
> >
> > Ultimately, the more I think about this, the more I believe that we're
> > adding noise to the API (with the new overloaded variant of send) for a
> > feature that will likely bring confusion and even frustration to anyone
> > besides maintainers of Kafka Streams who tries to use it.
> >
> > If the only concern about covering broker-side errors is that it would be
> > more difficult to implement, I believe we should strongly reconsider that
> > alternative. That said, if there is a straightforward way to explain this
> > feature to new users that won't mislead them or require them to do
> research
> > on producer internals, then I can still live with it.
> >
> > Regarding a list of recoverable vs. irrecoverable errors, this is
> actually
> > the subject of another recently-introduced KIP:
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-1050%3A+Consistent+error+handling+for+Transactions
> >
> > Finally, I'd also like to ask the people who have already voted (Andrew,
> > Matthias) if, at the time they voted, they believed that the API would
> > handle all errors, or only the subset of errors that would cause a record
> > to be rejected from a batch before it can be sent to a broker.
> >
> > Best,
> >
> > Chris
> >
> > On Thu, Jul 4, 2024 at 12:43 PM Alieh Saeedi
> 
> > wrote:
> >
> >> Salut from the KIP’s author
> >>
> >>
> >> Clarifying two points:
> >>
> >>
> >> 1) broker side errors:
> >>
> >> As far as I remember we are not going to cover the errors originating
> from
> >> the broker!
> >>
> >> A historical fact: One of the debate points in KIP-1038 was that by
> >> defining a producer custom handler, the user may assume that broker-side
> >> errors must be covered as well. They may define a handler for handling
> >> `Re

Re: [DISCUSS] KIP-1059: Enable the Producer flush() method to clear the latest send() error

2024-07-09 Thread Andrew Schofield
Hi Chris,
As it stands, the error handling for transactions in KafkaProducer is not
ideal. There’s no reason why a failed operation should fail a transaction
provided that the application can tell that the operation was not included
in the transaction and then make its own decision whether to continue or
back out. So, I think I disagree with the original premise of a client-side
error state for a transaction, but we are where we are.

When I voted, I did not expect the KIP to handle ALL errors which could
conceivably be handled. I did expect it to handle client-side send errors
that would cause a record to be rejected from a batch before sending to a
broker. I think that it does make the KafkaProducer interface very slightly
more complicated, but the new option is a clear improvement and I
don’t see anyone getting into a mess using it.

I think broker-side errors are more tricky and I don’t think an overload
on the send() method is going to do the job. I don’t see that as a problem
with the KIP, just that the underlying RPCs and behaviour is not very
amenable to record-specific error handling. The Produce RPC is a
complicated beast which can include a set of records for mutiple
topic-partitions. Although ProduceResponse v10 does include record
errors, I don’t believe this is surfaced in the client. Let’s imagine something
like broker-side record validation which barfs on one record. Failing an
entire batch is easier, but less useful if the problem is related to one record.

In summary, I’m happy that my vote stands, and I am happy with the KIP
only supporting client-side record errors.

Thanks,
Andrew

> On 8 Jul 2024, at 16:37, Chris Egerton  wrote:
>
> Hi Alieh,
>
> Can you clarify why broker-side errors shouldn't be covered? The only real
> rationale I can come up with is that it's easier to implement.
>
> "Things were better for Kafka Streams before KAFKA-9279 was fixed" isn't
> very convincing, because Kafka Streams is not the only user of the Java
> producer client. And for others, especially new users, I doubt that this
> new API we're proposing would make sense without having to consult a lot of
> historical context.
>
> I also don't think that most users will know or even care about the
> distinction between errors that cause a record to fail before it's added to
> a batch vs. after. If you were writing a producer application of your own,
> and you wanted to handle RecordTooLargeException instances by dropping a
> record without aborting a transaction, would you care about whether it was
> your client or your broker that balked? Would you be happy if you wrote
> logic expecting that that problem was solved once and for all, only to
> learn that it could still affect you in other circumstances? Or,
> alternatively, would you be happy if you wanted to solve that problem and
> found an API that seemed to do exactly what you wanted, but after reading
> the fine print, realized you'd have to do it yourself instead?
>
> Ultimately, the more I think about this, the more I believe that we're
> adding noise to the API (with the new overloaded variant of send) for a
> feature that will likely bring confusion and even frustration to anyone
> besides maintainers of Kafka Streams who tries to use it.
>
> If the only concern about covering broker-side errors is that it would be
> more difficult to implement, I believe we should strongly reconsider that
> alternative. That said, if there is a straightforward way to explain this
> feature to new users that won't mislead them or require them to do research
> on producer internals, then I can still live with it.
>
> Regarding a list of recoverable vs. irrecoverable errors, this is actually
> the subject of another recently-introduced KIP:
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-1050%3A+Consistent+error+handling+for+Transactions
>
> Finally, I'd also like to ask the people who have already voted (Andrew,
> Matthias) if, at the time they voted, they believed that the API would
> handle all errors, or only the subset of errors that would cause a record
> to be rejected from a batch before it can be sent to a broker.
>
> Best,
>
> Chris
>
> On Thu, Jul 4, 2024 at 12:43 PM Alieh Saeedi 
> wrote:
>
>> Salut from the KIP’s author
>>
>>
>> Clarifying two points:
>>
>>
>> 1) broker side errors:
>>
>> As far as I remember we are not going to cover the errors originating from
>> the broker!
>>
>> A historical fact: One of the debate points in KIP-1038 was that by
>> defining a producer custom handler, the user may assume that broker-side
>> errors must be covered as well. They may define a handler for handling
>> `RecordTooLargeException` and still see such errors not being handled as
>> they wish.
>>
>>
>> 2) Regarding irrecoverable/recoverable errors:
>>
>> Before the fix of `KAFKA-9279`,  errors such as `RecordTooLargeException`
>> or errors related to missing meta data (both originating from Producer
>> `send()`) were considered as recoverable but after th

Re: [DISCUSS] KIP-1059: Enable the Producer flush() method to clear the latest send() error

2024-07-08 Thread Matthias J. Sax

Hey,

just catching up on the latest discussion, and I think we are going in 
circles? And maybe over-complicate things a little bit too much?



My take on the KIP is a follows:

 - As a minimum, we want to solve client side RecordTooLargeException

 - If we can tackle other client side errors "for free", why not 
include them? I am ok with either approach to either list all errors we 
cover or list all errors we want to exclude. If we cannot reach a quick 
consensus on this question, I would pragmatically limit it to 
RecordTooLargeException just to unblock the KIP and to address an issue 
users did hit (we can also update the enum to express this clearly, 
following a "list all error we cover explicitly" approach) -- We can do 
follow up KIP for other errors on an on-demand basis and fix-forward / 
enlarge the scope successively.


(The question what "poison poll" means might not have a precise answer, 
but a single too large record is for sure a poison pill...)



 - The KIP is pretty clear about "send" errors, not TX errors

 - The KIP was a little bit unspecific about client vs broker side 
errors, but Alieh clearly express on the DISCUSS thread that she intends 
to only address client side error, so I would propose to just stick with 
this for now (if we want to cover broker side error, we can always do a 
follow up KIP)


 - Chris, you mentioned to you don't want us to get into a corner that 
we cannot get out with a follow up KIP. This is basically always a risk, 
that I think we just need to accept. If Alieh wants to limit the scope 
to client side errors it would be a good incremental step forward and I 
think we should take it. Or is there any justification why we _need_ to 
expand the scope to broker side error (I agree, it would nice to 
include, but I don't see it as a must, and it seems it raises too many 
open questions, preventing us to make progress...)


Btw: I also find the argument of "it's much more difficult to implement" 
totally valid to limit the scope of some work... we do this ALL the time...



 - The proposed API makes sense (as I already expressed previously); 
using a `send()` overload and an expressive name for the enum and clear 
documentation, make the semantics and usage pattern easy to understand




-Matthias



On 7/8/24 12:44 PM, Alieh Saeedi wrote:

Hey Chris


Thanks for the comprehensive answer.


I don’t believe that things were better/worse before/after the fix of X/Y.
I strongly believe that things get better and better when we intend to
improve our current solutions. I am happy that experts, including you, are
so cautious and, at the same time so helpful with the changes and
improvements.


Regarding considering broker side errors: As you mentioned, it’s a bit
difficult since we must overload both `send` and `commitTnx`. It won’t be
doable only in `commitTnx` (see this
 please).
Also, the broker and the producer have different configurations and
consequently different constraints for a too large record. Long story
short, the conclusion is what you mentioned: that it will be harder to
implement and even to clarify for the user in my opinion (the user must set
`ignore-send-errors` in both methods, and otherwise we throw exceptions or
if they use it only in `send` we do action X and if they use it only in
`commitTnx` we do action Y, and so on).


Regarding clarifying the new `send` for users: as you suggested before, we
must use a reasonable heuristic that would cover the errors (without
explicitly list them). We can say something like: “setting
`ignore-send-errors`, prevents failing a transaction because of single
poison pill records that violated producer configs. The faulty record is
omitted from the batch and the transaction is committed with the rest of
the batch. The broker can still reject committing a transaction due to its
own rules/constraints.”

The main idea (as you called it heuristic) must be around the idea of
non-existing records in the batch. Also, explicitly mentioning `Producer`
configs helps not assuming broker-side error coverage. This is for too
large records. Of course, we can re-word it and add the other cases as well
to make it more accurate.


About the current votes: I assume that both the new PR
 (changes in `send`) and the
former PR  (changes in
`commitTnx`) do not consider broker-side errors. But of course it was a
main concern of all experts (specifically Andrew) since KIP-1038 and
through the discussions we came to this agreement. BTW, it is my
opinion/assumption and might be wrong.


Cheers,

Alieh

On Mon, Jul 8, 2024 at 5:38 PM Chris Egerton 
wrote:


Hi Alieh,

Can you clarify why broker-side errors shouldn't be covered? The only real
rationale I can come up with is that it's easier to implement.

"Things were better for Kafka Streams before KAFKA-9279 was fixed" isn't
very 

Re: [DISCUSS] KIP-1059: Enable the Producer flush() method to clear the latest send() error

2024-07-08 Thread Alieh Saeedi
Hey Chris


Thanks for the comprehensive answer.


I don’t believe that things were better/worse before/after the fix of X/Y.
I strongly believe that things get better and better when we intend to
improve our current solutions. I am happy that experts, including you, are
so cautious and, at the same time so helpful with the changes and
improvements.


Regarding considering broker side errors: As you mentioned, it’s a bit
difficult since we must overload both `send` and `commitTnx`. It won’t be
doable only in `commitTnx` (see this
 please).
Also, the broker and the producer have different configurations and
consequently different constraints for a too large record. Long story
short, the conclusion is what you mentioned: that it will be harder to
implement and even to clarify for the user in my opinion (the user must set
`ignore-send-errors` in both methods, and otherwise we throw exceptions or
if they use it only in `send` we do action X and if they use it only in
`commitTnx` we do action Y, and so on).


Regarding clarifying the new `send` for users: as you suggested before, we
must use a reasonable heuristic that would cover the errors (without
explicitly list them). We can say something like: “setting
`ignore-send-errors`, prevents failing a transaction because of single
poison pill records that violated producer configs. The faulty record is
omitted from the batch and the transaction is committed with the rest of
the batch. The broker can still reject committing a transaction due to its
own rules/constraints.”

The main idea (as you called it heuristic) must be around the idea of
non-existing records in the batch. Also, explicitly mentioning `Producer`
configs helps not assuming broker-side error coverage. This is for too
large records. Of course, we can re-word it and add the other cases as well
to make it more accurate.


About the current votes: I assume that both the new PR
 (changes in `send`) and the
former PR  (changes in
`commitTnx`) do not consider broker-side errors. But of course it was a
main concern of all experts (specifically Andrew) since KIP-1038 and
through the discussions we came to this agreement. BTW, it is my
opinion/assumption and might be wrong.


Cheers,

Alieh

On Mon, Jul 8, 2024 at 5:38 PM Chris Egerton 
wrote:

> Hi Alieh,
>
> Can you clarify why broker-side errors shouldn't be covered? The only real
> rationale I can come up with is that it's easier to implement.
>
> "Things were better for Kafka Streams before KAFKA-9279 was fixed" isn't
> very convincing, because Kafka Streams is not the only user of the Java
> producer client. And for others, especially new users, I doubt that this
> new API we're proposing would make sense without having to consult a lot of
> historical context.
>
> I also don't think that most users will know or even care about the
> distinction between errors that cause a record to fail before it's added to
> a batch vs. after. If you were writing a producer application of your own,
> and you wanted to handle RecordTooLargeException instances by dropping a
> record without aborting a transaction, would you care about whether it was
> your client or your broker that balked? Would you be happy if you wrote
> logic expecting that that problem was solved once and for all, only to
> learn that it could still affect you in other circumstances? Or,
> alternatively, would you be happy if you wanted to solve that problem and
> found an API that seemed to do exactly what you wanted, but after reading
> the fine print, realized you'd have to do it yourself instead?
>
> Ultimately, the more I think about this, the more I believe that we're
> adding noise to the API (with the new overloaded variant of send) for a
> feature that will likely bring confusion and even frustration to anyone
> besides maintainers of Kafka Streams who tries to use it.
>
> If the only concern about covering broker-side errors is that it would be
> more difficult to implement, I believe we should strongly reconsider that
> alternative. That said, if there is a straightforward way to explain this
> feature to new users that won't mislead them or require them to do research
> on producer internals, then I can still live with it.
>
> Regarding a list of recoverable vs. irrecoverable errors, this is actually
> the subject of another recently-introduced KIP:
>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-1050%3A+Consistent+error+handling+for+Transactions
>
> Finally, I'd also like to ask the people who have already voted (Andrew,
> Matthias) if, at the time they voted, they believed that the API would
> handle all errors, or only the subset of errors that would cause a record
> to be rejected from a batch before it can be sent to a broker.
>
> Best,
>
> Chris
>
> On Thu, Jul 4, 2024 at 12:43 PM Alieh Saeedi  >
> wrote:
>
> > Salut f

Re: [DISCUSS] KIP-1059: Enable the Producer flush() method to clear the latest send() error

2024-07-08 Thread Chris Egerton
Hi Alieh,

Can you clarify why broker-side errors shouldn't be covered? The only real
rationale I can come up with is that it's easier to implement.

"Things were better for Kafka Streams before KAFKA-9279 was fixed" isn't
very convincing, because Kafka Streams is not the only user of the Java
producer client. And for others, especially new users, I doubt that this
new API we're proposing would make sense without having to consult a lot of
historical context.

I also don't think that most users will know or even care about the
distinction between errors that cause a record to fail before it's added to
a batch vs. after. If you were writing a producer application of your own,
and you wanted to handle RecordTooLargeException instances by dropping a
record without aborting a transaction, would you care about whether it was
your client or your broker that balked? Would you be happy if you wrote
logic expecting that that problem was solved once and for all, only to
learn that it could still affect you in other circumstances? Or,
alternatively, would you be happy if you wanted to solve that problem and
found an API that seemed to do exactly what you wanted, but after reading
the fine print, realized you'd have to do it yourself instead?

Ultimately, the more I think about this, the more I believe that we're
adding noise to the API (with the new overloaded variant of send) for a
feature that will likely bring confusion and even frustration to anyone
besides maintainers of Kafka Streams who tries to use it.

If the only concern about covering broker-side errors is that it would be
more difficult to implement, I believe we should strongly reconsider that
alternative. That said, if there is a straightforward way to explain this
feature to new users that won't mislead them or require them to do research
on producer internals, then I can still live with it.

Regarding a list of recoverable vs. irrecoverable errors, this is actually
the subject of another recently-introduced KIP:
https://cwiki.apache.org/confluence/display/KAFKA/KIP-1050%3A+Consistent+error+handling+for+Transactions

Finally, I'd also like to ask the people who have already voted (Andrew,
Matthias) if, at the time they voted, they believed that the API would
handle all errors, or only the subset of errors that would cause a record
to be rejected from a batch before it can be sent to a broker.

Best,

Chris

On Thu, Jul 4, 2024 at 12:43 PM Alieh Saeedi 
wrote:

> Salut from the KIP’s author
>
>
> Clarifying two points:
>
>
> 1) broker side errors:
>
> As far as I remember we are not going to cover the errors originating from
> the broker!
>
> A historical fact: One of the debate points in KIP-1038 was that by
> defining a producer custom handler, the user may assume that broker-side
> errors must be covered as well. They may define a handler for handling
> `RecordTooLargeException` and still see such errors not being handled as
> they wish.
>
>
> 2) Regarding irrecoverable/recoverable errors:
>
> Before the fix of `KAFKA-9279`,  errors such as `RecordTooLargeException`
> or errors related to missing meta data (both originating from Producer
> `send()`) were considered as recoverable but after that they turned into
> being irrecoverable without changing any Javadocs or having any KIP.  All
> the effort made in this KIP and the former one have been towards returning
> to the former state.
>
>
> I am sure that it is clear for you that which sort of errors we are going
> to cover: A single record may happen to NOT get added to the batch due to
> the issues with the record or its corresponding topic. The point was that
> if the record is not added to the batch let ’s don’t fail the whole batch
> because of that non-existing record. We never intended to do sth in broker
> side or ignore more important errors.  But I agree with you Chris. If we
> are adding a new API, we must have good documentation for that. The
> sentence `all irrecoverable transactional errors will still be fatal` as
> you suggested is good. What do you think? I am totally against enumerating
> errors in Javadocs since these sort of errors can be changing during
> time.  More
> over, have you ever seen any list of recoverable or irrecoverable errors
> somewhere so far?
>
>
> Bests,
>
> Alieh
>
> On Wed, Jul 3, 2024 at 6:07 PM Chris Egerton 
> wrote:
>
> > Hi Justine,
> >
> > I agree that enumerating a list of errors that should be covered by the
> KIP
> > is difficult; I was thinking it might be easier if we list the errors
> that
> > should _not_ be covered by the KIP, and only if we can't define a
> > reasonable heuristic that would cover them without having to explicitly
> > list them. Could it be enough to say "all irrecoverable transactional
> > errors will still be fatal", or even just "all transactional errors (as
> > opposed to errors related to this specific record) will still be fatal"?
> >
> > Cheers,
> >
> > Chris
> >
> > On Wed, Jul 3, 2024 at 11:56 AM Justine Olshan
> > 
> > w

Re: [DISCUSS] KIP-1059: Enable the Producer flush() method to clear the latest send() error

2024-07-04 Thread Alieh Saeedi
Salut from the KIP’s author


Clarifying two points:


1) broker side errors:

As far as I remember we are not going to cover the errors originating from
the broker!

A historical fact: One of the debate points in KIP-1038 was that by
defining a producer custom handler, the user may assume that broker-side
errors must be covered as well. They may define a handler for handling
`RecordTooLargeException` and still see such errors not being handled as
they wish.


2) Regarding irrecoverable/recoverable errors:

Before the fix of `KAFKA-9279`,  errors such as `RecordTooLargeException`
or errors related to missing meta data (both originating from Producer
`send()`) were considered as recoverable but after that they turned into
being irrecoverable without changing any Javadocs or having any KIP.  All
the effort made in this KIP and the former one have been towards returning
to the former state.


I am sure that it is clear for you that which sort of errors we are going
to cover: A single record may happen to NOT get added to the batch due to
the issues with the record or its corresponding topic. The point was that
if the record is not added to the batch let ’s don’t fail the whole batch
because of that non-existing record. We never intended to do sth in broker
side or ignore more important errors.  But I agree with you Chris. If we
are adding a new API, we must have good documentation for that. The
sentence `all irrecoverable transactional errors will still be fatal` as
you suggested is good. What do you think? I am totally against enumerating
errors in Javadocs since these sort of errors can be changing during
time.  More
over, have you ever seen any list of recoverable or irrecoverable errors
somewhere so far?


Bests,

Alieh

On Wed, Jul 3, 2024 at 6:07 PM Chris Egerton 
wrote:

> Hi Justine,
>
> I agree that enumerating a list of errors that should be covered by the KIP
> is difficult; I was thinking it might be easier if we list the errors that
> should _not_ be covered by the KIP, and only if we can't define a
> reasonable heuristic that would cover them without having to explicitly
> list them. Could it be enough to say "all irrecoverable transactional
> errors will still be fatal", or even just "all transactional errors (as
> opposed to errors related to this specific record) will still be fatal"?
>
> Cheers,
>
> Chris
>
> On Wed, Jul 3, 2024 at 11:56 AM Justine Olshan
> 
> wrote:
>
> > Hey Chris,
> >
> > I think what you say makes sense. I agree that defining the behavior
> based
> > on code that can possibly change is not a good idea, and I was trying to
> > get a clearer definition from the KIP's author :)
> >
> > I think it can always be hard to ensure that only specific errors are
> > handled unless they are explicitly enumerated in code as the code can
> > change and can be changed by folks who are not aware of this KIP or
> > conversation.
> > I personally don't have the bandwidth to do this definition/enumeration
> of
> > errors, so hopefully Alieh can expand upon this.
> >
> > Justine
> >
> > On Wed, Jul 3, 2024 at 8:28 AM Chris Egerton 
> > wrote:
> >
> > > Hi Alieh,
> > >
> > > I don't love defining the changes for this KIP in terms of a catch
> clause
> > > in the KafkaProducer class, for two reasons. First, the set of errors
> > that
> > > are handled by that clause may shift over time as the code base is
> > > modified, and second, it would be fairly opaque to users who want to
> > > understand whether an error would be affected by using this API or not.
> > >
> > > It also seems strange that we'd handle some types of
> > > RecordTooLargeException (i.e., ones reported client-side) with this
> API,
> > > but not others (i.e., ones reported by a broker).
> > >
> > > I think this kind of API would be most powerful, most intuitive to
> users,
> > > and easiest to document if we expanded the scope to all
> > record-send-related
> > > errors, except anything indicating issues with exactly-once semantics.
> > That
> > > would include records that are too large (when caught both client- and
> > > server-side), records that can't be sent due to authorization failures,
> > > records sent to nonexistent topics/topic partitions, and keyless
> records
> > > sent to compacted topics. It would not include
> > > ProducerFencedException, InvalidProducerEpochException,
> > > UnsupportedVersionException,
> > > and possibly others.
> > >
> > > @Justine -- do you think it would be possible to develop either a
> better
> > > definition for the kinds of "excluded" errors that should not be
> covered
> > by
> > > this API, or, barring that, a comprehensive list of exact error types?
> > And
> > > do you think this would be acceptable in terms of risk and complexity?
> > >
> > > Cheers,
> > >
> > > Chris
> > >
> > > On Tue, Jul 2, 2024 at 5:05 PM Alieh Saeedi
>  > >
> > > wrote:
> > >
> > > > Hey Justine,
> > > >
> > > > About the consequences: the consequences will be like when we did not
> > > have
> > > > the fix

Re: [DISCUSS] KIP-1059: Enable the Producer flush() method to clear the latest send() error

2024-07-03 Thread Chris Egerton
Hi Justine,

I agree that enumerating a list of errors that should be covered by the KIP
is difficult; I was thinking it might be easier if we list the errors that
should _not_ be covered by the KIP, and only if we can't define a
reasonable heuristic that would cover them without having to explicitly
list them. Could it be enough to say "all irrecoverable transactional
errors will still be fatal", or even just "all transactional errors (as
opposed to errors related to this specific record) will still be fatal"?

Cheers,

Chris

On Wed, Jul 3, 2024 at 11:56 AM Justine Olshan 
wrote:

> Hey Chris,
>
> I think what you say makes sense. I agree that defining the behavior based
> on code that can possibly change is not a good idea, and I was trying to
> get a clearer definition from the KIP's author :)
>
> I think it can always be hard to ensure that only specific errors are
> handled unless they are explicitly enumerated in code as the code can
> change and can be changed by folks who are not aware of this KIP or
> conversation.
> I personally don't have the bandwidth to do this definition/enumeration of
> errors, so hopefully Alieh can expand upon this.
>
> Justine
>
> On Wed, Jul 3, 2024 at 8:28 AM Chris Egerton 
> wrote:
>
> > Hi Alieh,
> >
> > I don't love defining the changes for this KIP in terms of a catch clause
> > in the KafkaProducer class, for two reasons. First, the set of errors
> that
> > are handled by that clause may shift over time as the code base is
> > modified, and second, it would be fairly opaque to users who want to
> > understand whether an error would be affected by using this API or not.
> >
> > It also seems strange that we'd handle some types of
> > RecordTooLargeException (i.e., ones reported client-side) with this API,
> > but not others (i.e., ones reported by a broker).
> >
> > I think this kind of API would be most powerful, most intuitive to users,
> > and easiest to document if we expanded the scope to all
> record-send-related
> > errors, except anything indicating issues with exactly-once semantics.
> That
> > would include records that are too large (when caught both client- and
> > server-side), records that can't be sent due to authorization failures,
> > records sent to nonexistent topics/topic partitions, and keyless records
> > sent to compacted topics. It would not include
> > ProducerFencedException, InvalidProducerEpochException,
> > UnsupportedVersionException,
> > and possibly others.
> >
> > @Justine -- do you think it would be possible to develop either a better
> > definition for the kinds of "excluded" errors that should not be covered
> by
> > this API, or, barring that, a comprehensive list of exact error types?
> And
> > do you think this would be acceptable in terms of risk and complexity?
> >
> > Cheers,
> >
> > Chris
> >
> > On Tue, Jul 2, 2024 at 5:05 PM Alieh Saeedi  >
> > wrote:
> >
> > > Hey Justine,
> > >
> > > About the consequences: the consequences will be like when we did not
> > have
> > > the fix made in `KAFKA-9279`: silent loss of data! Obviously, when the
> > user
> > > intentionally chose to ignore errors, that would not be silent any
> more.
> > > Right?
> > > Of course, considering all types of `ApiException`s would be too broad.
> > But
> > > are the exceptions caught in `catch(ApiException e)` of the `doSend()`
> > > method also too broad?
> > >
> > > -Alieh
> > >
> > > On Tue, Jul 2, 2024 at 9:45 PM Justine Olshan
> >  > > >
> > > wrote:
> > >
> > > > Hey Alieh,
> > > >
> > > > If we want to allow any error to be ignored we should probably run
> > > through
> > > > all the errors to make sure they make sense.
> > > > I just want to feel confident that we aren't just making a decision
> > > without
> > > > considering the consequences carefully.
> > > >
> > > > Justine
> > > >
> > > > On Tue, Jul 2, 2024 at 12:30 PM Alieh Saeedi
> > >  > > > >
> > > > wrote:
> > > >
> > > > > Hey Justine,
> > > > >
> > > > > yes we talked about `RecordTooLargeException` as an example, but
> did
> > we
> > > > > ever limit ourselves to only this specific exception? I think
> neither
> > > in
> > > > > the KIP nor in the PR.  As Chris mentioned, this KIP is going to
> undo
> > > > what
> > > > > we have done in `KAFKA-9279` in case 1) the user is in a
> transaction
> > > and
> > > > 2)
> > > > > he decides to ignore the errors in which the record was not even
> > added
> > > to
> > > > > the batch. Yes, and we suggested some methods for undoing or, in
> > fact,
> > > > > moving back the transaction from the error state in `flush` or in
> > > > > `commitTnx` and we finally came to the idea of not even doing the
> > > changes
> > > > > (better than undoing) in `send`.
> > > > >
> > > > > Bests,
> > > > > Alieh
> > > > >
> > > > > On Tue, Jul 2, 2024 at 8:03 PM Justine Olshan
> > > >  > > > > >
> > > > > wrote:
> > > > >
> > > > > > Hey folks,
> > > > > >
> > > > > > I understand where you are coming from by asking for specific use
> > > > cases.
> > > >

Re: [DISCUSS] KIP-1059: Enable the Producer flush() method to clear the latest send() error

2024-07-03 Thread Justine Olshan
Hey Chris,

I think what you say makes sense. I agree that defining the behavior based
on code that can possibly change is not a good idea, and I was trying to
get a clearer definition from the KIP's author :)

I think it can always be hard to ensure that only specific errors are
handled unless they are explicitly enumerated in code as the code can
change and can be changed by folks who are not aware of this KIP or
conversation.
I personally don't have the bandwidth to do this definition/enumeration of
errors, so hopefully Alieh can expand upon this.

Justine

On Wed, Jul 3, 2024 at 8:28 AM Chris Egerton 
wrote:

> Hi Alieh,
>
> I don't love defining the changes for this KIP in terms of a catch clause
> in the KafkaProducer class, for two reasons. First, the set of errors that
> are handled by that clause may shift over time as the code base is
> modified, and second, it would be fairly opaque to users who want to
> understand whether an error would be affected by using this API or not.
>
> It also seems strange that we'd handle some types of
> RecordTooLargeException (i.e., ones reported client-side) with this API,
> but not others (i.e., ones reported by a broker).
>
> I think this kind of API would be most powerful, most intuitive to users,
> and easiest to document if we expanded the scope to all record-send-related
> errors, except anything indicating issues with exactly-once semantics. That
> would include records that are too large (when caught both client- and
> server-side), records that can't be sent due to authorization failures,
> records sent to nonexistent topics/topic partitions, and keyless records
> sent to compacted topics. It would not include
> ProducerFencedException, InvalidProducerEpochException,
> UnsupportedVersionException,
> and possibly others.
>
> @Justine -- do you think it would be possible to develop either a better
> definition for the kinds of "excluded" errors that should not be covered by
> this API, or, barring that, a comprehensive list of exact error types? And
> do you think this would be acceptable in terms of risk and complexity?
>
> Cheers,
>
> Chris
>
> On Tue, Jul 2, 2024 at 5:05 PM Alieh Saeedi 
> wrote:
>
> > Hey Justine,
> >
> > About the consequences: the consequences will be like when we did not
> have
> > the fix made in `KAFKA-9279`: silent loss of data! Obviously, when the
> user
> > intentionally chose to ignore errors, that would not be silent any more.
> > Right?
> > Of course, considering all types of `ApiException`s would be too broad.
> But
> > are the exceptions caught in `catch(ApiException e)` of the `doSend()`
> > method also too broad?
> >
> > -Alieh
> >
> > On Tue, Jul 2, 2024 at 9:45 PM Justine Olshan
>  > >
> > wrote:
> >
> > > Hey Alieh,
> > >
> > > If we want to allow any error to be ignored we should probably run
> > through
> > > all the errors to make sure they make sense.
> > > I just want to feel confident that we aren't just making a decision
> > without
> > > considering the consequences carefully.
> > >
> > > Justine
> > >
> > > On Tue, Jul 2, 2024 at 12:30 PM Alieh Saeedi
> >  > > >
> > > wrote:
> > >
> > > > Hey Justine,
> > > >
> > > > yes we talked about `RecordTooLargeException` as an example, but did
> we
> > > > ever limit ourselves to only this specific exception? I think neither
> > in
> > > > the KIP nor in the PR.  As Chris mentioned, this KIP is going to undo
> > > what
> > > > we have done in `KAFKA-9279` in case 1) the user is in a transaction
> > and
> > > 2)
> > > > he decides to ignore the errors in which the record was not even
> added
> > to
> > > > the batch. Yes, and we suggested some methods for undoing or, in
> fact,
> > > > moving back the transaction from the error state in `flush` or in
> > > > `commitTnx` and we finally came to the idea of not even doing the
> > changes
> > > > (better than undoing) in `send`.
> > > >
> > > > Bests,
> > > > Alieh
> > > >
> > > > On Tue, Jul 2, 2024 at 8:03 PM Justine Olshan
> > >  > > > >
> > > > wrote:
> > > >
> > > > > Hey folks,
> > > > >
> > > > > I understand where you are coming from by asking for specific use
> > > cases.
> > > > My
> > > > > understanding based on previous conversations was that there were a
> > few
> > > > > different errors that have been seen.
> > > > > One example I heard some information about was when the record was
> > too
> > > > > large and it fails the batch. Besides that, I'm not really sure if
> > > there
> > > > > are cases in mind, though it is fair to ask on those and bring them
> > up.
> > > > >
> > > > > > Does a record qualify as a poison pill if it targets a topic that
> > > > > doesn't exist? Or if it targets a topic that the producer principal
> > > lacks
> > > > > ACLs for? What if it fails broker-side validation (e.g., has a null
> > key
> > > > for
> > > > > a compacted topic)?
> > > > >
> > > > > I think there was some parallel work with addressing the
> > > > > UnknownTopicOrPartitionError in another way. As for the othe

Re: [DISCUSS] KIP-1059: Enable the Producer flush() method to clear the latest send() error

2024-07-03 Thread Chris Egerton
Hi Alieh,

I don't love defining the changes for this KIP in terms of a catch clause
in the KafkaProducer class, for two reasons. First, the set of errors that
are handled by that clause may shift over time as the code base is
modified, and second, it would be fairly opaque to users who want to
understand whether an error would be affected by using this API or not.

It also seems strange that we'd handle some types of
RecordTooLargeException (i.e., ones reported client-side) with this API,
but not others (i.e., ones reported by a broker).

I think this kind of API would be most powerful, most intuitive to users,
and easiest to document if we expanded the scope to all record-send-related
errors, except anything indicating issues with exactly-once semantics. That
would include records that are too large (when caught both client- and
server-side), records that can't be sent due to authorization failures,
records sent to nonexistent topics/topic partitions, and keyless records
sent to compacted topics. It would not include
ProducerFencedException, InvalidProducerEpochException,
UnsupportedVersionException,
and possibly others.

@Justine -- do you think it would be possible to develop either a better
definition for the kinds of "excluded" errors that should not be covered by
this API, or, barring that, a comprehensive list of exact error types? And
do you think this would be acceptable in terms of risk and complexity?

Cheers,

Chris

On Tue, Jul 2, 2024 at 5:05 PM Alieh Saeedi 
wrote:

> Hey Justine,
>
> About the consequences: the consequences will be like when we did not have
> the fix made in `KAFKA-9279`: silent loss of data! Obviously, when the user
> intentionally chose to ignore errors, that would not be silent any more.
> Right?
> Of course, considering all types of `ApiException`s would be too broad. But
> are the exceptions caught in `catch(ApiException e)` of the `doSend()`
> method also too broad?
>
> -Alieh
>
> On Tue, Jul 2, 2024 at 9:45 PM Justine Olshan  >
> wrote:
>
> > Hey Alieh,
> >
> > If we want to allow any error to be ignored we should probably run
> through
> > all the errors to make sure they make sense.
> > I just want to feel confident that we aren't just making a decision
> without
> > considering the consequences carefully.
> >
> > Justine
> >
> > On Tue, Jul 2, 2024 at 12:30 PM Alieh Saeedi
>  > >
> > wrote:
> >
> > > Hey Justine,
> > >
> > > yes we talked about `RecordTooLargeException` as an example, but did we
> > > ever limit ourselves to only this specific exception? I think neither
> in
> > > the KIP nor in the PR.  As Chris mentioned, this KIP is going to undo
> > what
> > > we have done in `KAFKA-9279` in case 1) the user is in a transaction
> and
> > 2)
> > > he decides to ignore the errors in which the record was not even added
> to
> > > the batch. Yes, and we suggested some methods for undoing or, in fact,
> > > moving back the transaction from the error state in `flush` or in
> > > `commitTnx` and we finally came to the idea of not even doing the
> changes
> > > (better than undoing) in `send`.
> > >
> > > Bests,
> > > Alieh
> > >
> > > On Tue, Jul 2, 2024 at 8:03 PM Justine Olshan
> >  > > >
> > > wrote:
> > >
> > > > Hey folks,
> > > >
> > > > I understand where you are coming from by asking for specific use
> > cases.
> > > My
> > > > understanding based on previous conversations was that there were a
> few
> > > > different errors that have been seen.
> > > > One example I heard some information about was when the record was
> too
> > > > large and it fails the batch. Besides that, I'm not really sure if
> > there
> > > > are cases in mind, though it is fair to ask on those and bring them
> up.
> > > >
> > > > > Does a record qualify as a poison pill if it targets a topic that
> > > > doesn't exist? Or if it targets a topic that the producer principal
> > lacks
> > > > ACLs for? What if it fails broker-side validation (e.g., has a null
> key
> > > for
> > > > a compacted topic)?
> > > >
> > > > I think there was some parallel work with addressing the
> > > > UnknownTopicOrPartitionError in another way. As for the other checks,
> > > acls,
> > > > validation etc. I am not aware of that being in Alieh's scope, but we
> > > > should be clear about exactly what we are doing.
> > > >
> > > > All errors that fall into ApiException seems too broad to me.
> > > >
> > > > Justine
> > > >
> > > > On Tue, Jul 2, 2024 at 10:51 AM Alieh Saeedi
> > >  > > > >
> > > > wrote:
> > > >
> > > > > Hey Chris,
> > > > > thanks for sharing your concerns.
> > > > >
> > > > > 1) About the language of KIP (or maybe later in Javadocs): Is that
> > > > alright
> > > > > if I write all errors that fall into the `ApiException` category
> > thrown
> > > > > (actually returned) by Producer?
> > > > > 2) About future expansion: do you have any better suggestions for
> > enum
> > > > > names? Do you think `IGNORE_API_EXEPTIONS` or something like that
> is
> > a
> > > > > "better/more accurate" one

Re: [DISCUSS] KIP-1059: Enable the Producer flush() method to clear the latest send() error

2024-07-02 Thread Alieh Saeedi
Hey Justine,

About the consequences: the consequences will be like when we did not have
the fix made in `KAFKA-9279`: silent loss of data! Obviously, when the user
intentionally chose to ignore errors, that would not be silent any more.
Right?
Of course, considering all types of `ApiException`s would be too broad. But
are the exceptions caught in `catch(ApiException e)` of the `doSend()`
method also too broad?

-Alieh

On Tue, Jul 2, 2024 at 9:45 PM Justine Olshan 
wrote:

> Hey Alieh,
>
> If we want to allow any error to be ignored we should probably run through
> all the errors to make sure they make sense.
> I just want to feel confident that we aren't just making a decision without
> considering the consequences carefully.
>
> Justine
>
> On Tue, Jul 2, 2024 at 12:30 PM Alieh Saeedi  >
> wrote:
>
> > Hey Justine,
> >
> > yes we talked about `RecordTooLargeException` as an example, but did we
> > ever limit ourselves to only this specific exception? I think neither in
> > the KIP nor in the PR.  As Chris mentioned, this KIP is going to undo
> what
> > we have done in `KAFKA-9279` in case 1) the user is in a transaction and
> 2)
> > he decides to ignore the errors in which the record was not even added to
> > the batch. Yes, and we suggested some methods for undoing or, in fact,
> > moving back the transaction from the error state in `flush` or in
> > `commitTnx` and we finally came to the idea of not even doing the changes
> > (better than undoing) in `send`.
> >
> > Bests,
> > Alieh
> >
> > On Tue, Jul 2, 2024 at 8:03 PM Justine Olshan
>  > >
> > wrote:
> >
> > > Hey folks,
> > >
> > > I understand where you are coming from by asking for specific use
> cases.
> > My
> > > understanding based on previous conversations was that there were a few
> > > different errors that have been seen.
> > > One example I heard some information about was when the record was too
> > > large and it fails the batch. Besides that, I'm not really sure if
> there
> > > are cases in mind, though it is fair to ask on those and bring them up.
> > >
> > > > Does a record qualify as a poison pill if it targets a topic that
> > > doesn't exist? Or if it targets a topic that the producer principal
> lacks
> > > ACLs for? What if it fails broker-side validation (e.g., has a null key
> > for
> > > a compacted topic)?
> > >
> > > I think there was some parallel work with addressing the
> > > UnknownTopicOrPartitionError in another way. As for the other checks,
> > acls,
> > > validation etc. I am not aware of that being in Alieh's scope, but we
> > > should be clear about exactly what we are doing.
> > >
> > > All errors that fall into ApiException seems too broad to me.
> > >
> > > Justine
> > >
> > > On Tue, Jul 2, 2024 at 10:51 AM Alieh Saeedi
> >  > > >
> > > wrote:
> > >
> > > > Hey Chris,
> > > > thanks for sharing your concerns.
> > > >
> > > > 1) About the language of KIP (or maybe later in Javadocs): Is that
> > > alright
> > > > if I write all errors that fall into the `ApiException` category
> thrown
> > > > (actually returned) by Producer?
> > > > 2) About future expansion: do you have any better suggestions for
> enum
> > > > names? Do you think `IGNORE_API_EXEPTIONS` or something like that is
> a
> > > > "better/more accurate" one?
> > > >
> > > > Bests,
> > > > Alieh
> > > >
> > > > On Tue, Jul 2, 2024 at 7:29 PM Chris Egerton  >
> > > > wrote:
> > > >
> > > > > Hi Alieh and Justine,
> > > > >
> > > > > I'm concerned that we're settling on a definition of "poison pill"
> > > that's
> > > > > easiest to tackle right now but may lead to shortcomings down the
> > > road. I
> > > > > understand the relationship between this KIP and KAFKA-9279, and I
> > can
> > > > > totally get behind the desire to keep things small, focused, and
> > simple
> > > > in
> > > > > the name of avoiding bugs. However, what I don't think is clear at
> > all
> > > is
> > > > > what the "specific circumstances" are that Justine mentioned. I
> had a
> > > > > drastically different idea of what the intended behavioral change
> > would
> > > > be
> > > > > before looking at the draft PR.
> > > > >
> > > > > I would like 1) for us to be clearer about the categories of errors
> > > that
> > > > we
> > > > > want to cover with this new API (especially since we'll have to
> find
> > a
> > > > > clear, succinct way to document this for users), and 2) to make
> sure
> > > that
> > > > > if we do try to expand this API in the future, that we won't be
> > painted
> > > > > into a corner.
> > > > >
> > > > > For item 1, hopefully we can agree that the language in the KIP
> > > > > for IGNORE_SEND_ERRORS ("The records causing irrecoverable errors
> are
> > > > > excluded from the batch and the transaction is committed
> > > successfully.")
> > > > is
> > > > > pretty vague. If we start using the phrase "poison pill record"
> that
> > > > could
> > > > > help, but IMO more detail would still be needed. We know that we
> want
> > > to
> > > > > include records 

Re: [DISCUSS] KIP-1059: Enable the Producer flush() method to clear the latest send() error

2024-07-02 Thread Justine Olshan
Hey Alieh,

If we want to allow any error to be ignored we should probably run through
all the errors to make sure they make sense.
I just want to feel confident that we aren't just making a decision without
considering the consequences carefully.

Justine

On Tue, Jul 2, 2024 at 12:30 PM Alieh Saeedi 
wrote:

> Hey Justine,
>
> yes we talked about `RecordTooLargeException` as an example, but did we
> ever limit ourselves to only this specific exception? I think neither in
> the KIP nor in the PR.  As Chris mentioned, this KIP is going to undo what
> we have done in `KAFKA-9279` in case 1) the user is in a transaction and 2)
> he decides to ignore the errors in which the record was not even added to
> the batch. Yes, and we suggested some methods for undoing or, in fact,
> moving back the transaction from the error state in `flush` or in
> `commitTnx` and we finally came to the idea of not even doing the changes
> (better than undoing) in `send`.
>
> Bests,
> Alieh
>
> On Tue, Jul 2, 2024 at 8:03 PM Justine Olshan  >
> wrote:
>
> > Hey folks,
> >
> > I understand where you are coming from by asking for specific use cases.
> My
> > understanding based on previous conversations was that there were a few
> > different errors that have been seen.
> > One example I heard some information about was when the record was too
> > large and it fails the batch. Besides that, I'm not really sure if there
> > are cases in mind, though it is fair to ask on those and bring them up.
> >
> > > Does a record qualify as a poison pill if it targets a topic that
> > doesn't exist? Or if it targets a topic that the producer principal lacks
> > ACLs for? What if it fails broker-side validation (e.g., has a null key
> for
> > a compacted topic)?
> >
> > I think there was some parallel work with addressing the
> > UnknownTopicOrPartitionError in another way. As for the other checks,
> acls,
> > validation etc. I am not aware of that being in Alieh's scope, but we
> > should be clear about exactly what we are doing.
> >
> > All errors that fall into ApiException seems too broad to me.
> >
> > Justine
> >
> > On Tue, Jul 2, 2024 at 10:51 AM Alieh Saeedi
>  > >
> > wrote:
> >
> > > Hey Chris,
> > > thanks for sharing your concerns.
> > >
> > > 1) About the language of KIP (or maybe later in Javadocs): Is that
> > alright
> > > if I write all errors that fall into the `ApiException` category thrown
> > > (actually returned) by Producer?
> > > 2) About future expansion: do you have any better suggestions for enum
> > > names? Do you think `IGNORE_API_EXEPTIONS` or something like that is a
> > > "better/more accurate" one?
> > >
> > > Bests,
> > > Alieh
> > >
> > > On Tue, Jul 2, 2024 at 7:29 PM Chris Egerton 
> > > wrote:
> > >
> > > > Hi Alieh and Justine,
> > > >
> > > > I'm concerned that we're settling on a definition of "poison pill"
> > that's
> > > > easiest to tackle right now but may lead to shortcomings down the
> > road. I
> > > > understand the relationship between this KIP and KAFKA-9279, and I
> can
> > > > totally get behind the desire to keep things small, focused, and
> simple
> > > in
> > > > the name of avoiding bugs. However, what I don't think is clear at
> all
> > is
> > > > what the "specific circumstances" are that Justine mentioned. I had a
> > > > drastically different idea of what the intended behavioral change
> would
> > > be
> > > > before looking at the draft PR.
> > > >
> > > > I would like 1) for us to be clearer about the categories of errors
> > that
> > > we
> > > > want to cover with this new API (especially since we'll have to find
> a
> > > > clear, succinct way to document this for users), and 2) to make sure
> > that
> > > > if we do try to expand this API in the future, that we won't be
> painted
> > > > into a corner.
> > > >
> > > > For item 1, hopefully we can agree that the language in the KIP
> > > > for IGNORE_SEND_ERRORS ("The records causing irrecoverable errors are
> > > > excluded from the batch and the transaction is committed
> > successfully.")
> > > is
> > > > pretty vague. If we start using the phrase "poison pill record" that
> > > could
> > > > help, but IMO more detail would still be needed. We know that we want
> > to
> > > > include records that are so large that they can be immediately
> rejected
> > > by
> > > > the producer. But there are other cases that users might expect to be
> > > > handled. Does a record qualify as a poison pill if it targets a topic
> > > that
> > > > doesn't exist? Or if it targets a topic that the producer principal
> > lacks
> > > > ACLs for? What if it fails broker-side validation (e.g., has a null
> key
> > > for
> > > > a compacted topic)?
> > > >
> > > > For item 2, this really depends on how narrow the scope of what we're
> > > doing
> > > > right now is. If we only handle a subset of the examples I laid out
> > above
> > > > that could possibly be considered poison pills with this KIP, do we
> > want
> > > to
> > > > lock ourselves in to ne

Re: [DISCUSS] KIP-1059: Enable the Producer flush() method to clear the latest send() error

2024-07-02 Thread Alieh Saeedi
Hey Justine,

yes we talked about `RecordTooLargeException` as an example, but did we
ever limit ourselves to only this specific exception? I think neither in
the KIP nor in the PR.  As Chris mentioned, this KIP is going to undo what
we have done in `KAFKA-9279` in case 1) the user is in a transaction and 2)
he decides to ignore the errors in which the record was not even added to
the batch. Yes, and we suggested some methods for undoing or, in fact,
moving back the transaction from the error state in `flush` or in
`commitTnx` and we finally came to the idea of not even doing the changes
(better than undoing) in `send`.

Bests,
Alieh

On Tue, Jul 2, 2024 at 8:03 PM Justine Olshan 
wrote:

> Hey folks,
>
> I understand where you are coming from by asking for specific use cases. My
> understanding based on previous conversations was that there were a few
> different errors that have been seen.
> One example I heard some information about was when the record was too
> large and it fails the batch. Besides that, I'm not really sure if there
> are cases in mind, though it is fair to ask on those and bring them up.
>
> > Does a record qualify as a poison pill if it targets a topic that
> doesn't exist? Or if it targets a topic that the producer principal lacks
> ACLs for? What if it fails broker-side validation (e.g., has a null key for
> a compacted topic)?
>
> I think there was some parallel work with addressing the
> UnknownTopicOrPartitionError in another way. As for the other checks, acls,
> validation etc. I am not aware of that being in Alieh's scope, but we
> should be clear about exactly what we are doing.
>
> All errors that fall into ApiException seems too broad to me.
>
> Justine
>
> On Tue, Jul 2, 2024 at 10:51 AM Alieh Saeedi  >
> wrote:
>
> > Hey Chris,
> > thanks for sharing your concerns.
> >
> > 1) About the language of KIP (or maybe later in Javadocs): Is that
> alright
> > if I write all errors that fall into the `ApiException` category thrown
> > (actually returned) by Producer?
> > 2) About future expansion: do you have any better suggestions for enum
> > names? Do you think `IGNORE_API_EXEPTIONS` or something like that is a
> > "better/more accurate" one?
> >
> > Bests,
> > Alieh
> >
> > On Tue, Jul 2, 2024 at 7:29 PM Chris Egerton 
> > wrote:
> >
> > > Hi Alieh and Justine,
> > >
> > > I'm concerned that we're settling on a definition of "poison pill"
> that's
> > > easiest to tackle right now but may lead to shortcomings down the
> road. I
> > > understand the relationship between this KIP and KAFKA-9279, and I can
> > > totally get behind the desire to keep things small, focused, and simple
> > in
> > > the name of avoiding bugs. However, what I don't think is clear at all
> is
> > > what the "specific circumstances" are that Justine mentioned. I had a
> > > drastically different idea of what the intended behavioral change would
> > be
> > > before looking at the draft PR.
> > >
> > > I would like 1) for us to be clearer about the categories of errors
> that
> > we
> > > want to cover with this new API (especially since we'll have to find a
> > > clear, succinct way to document this for users), and 2) to make sure
> that
> > > if we do try to expand this API in the future, that we won't be painted
> > > into a corner.
> > >
> > > For item 1, hopefully we can agree that the language in the KIP
> > > for IGNORE_SEND_ERRORS ("The records causing irrecoverable errors are
> > > excluded from the batch and the transaction is committed
> successfully.")
> > is
> > > pretty vague. If we start using the phrase "poison pill record" that
> > could
> > > help, but IMO more detail would still be needed. We know that we want
> to
> > > include records that are so large that they can be immediately rejected
> > by
> > > the producer. But there are other cases that users might expect to be
> > > handled. Does a record qualify as a poison pill if it targets a topic
> > that
> > > doesn't exist? Or if it targets a topic that the producer principal
> lacks
> > > ACLs for? What if it fails broker-side validation (e.g., has a null key
> > for
> > > a compacted topic)?
> > >
> > > For item 2, this really depends on how narrow the scope of what we're
> > doing
> > > right now is. If we only handle a subset of the examples I laid out
> above
> > > that could possibly be considered poison pills with this KIP, do we
> want
> > to
> > > lock ourselves in to never addressing more in the future, or can we
> > choose
> > > an API (probably just enum names would be the only important decision
> > here)
> > > that leaves room for more later?
> > >
> > > Best,
> > >
> > > Chris
> > >
> > >
> > >
> > > On Tue, Jul 2, 2024 at 12:28 PM Justine Olshan
> > > 
> > > wrote:
> > >
> > > > Chris and Alieh,
> > > >
> > > > My understanding is that this KIP is really only trying to solve an
> > issue
> > > > of a "poison pill" record that fails send().
> > > > We've talked a lot about having a generic framework for all errors,
> 

Re: [DISCUSS] KIP-1059: Enable the Producer flush() method to clear the latest send() error

2024-07-02 Thread Justine Olshan
Hey folks,

I understand where you are coming from by asking for specific use cases. My
understanding based on previous conversations was that there were a few
different errors that have been seen.
One example I heard some information about was when the record was too
large and it fails the batch. Besides that, I'm not really sure if there
are cases in mind, though it is fair to ask on those and bring them up.

> Does a record qualify as a poison pill if it targets a topic that
doesn't exist? Or if it targets a topic that the producer principal lacks
ACLs for? What if it fails broker-side validation (e.g., has a null key for
a compacted topic)?

I think there was some parallel work with addressing the
UnknownTopicOrPartitionError in another way. As for the other checks, acls,
validation etc. I am not aware of that being in Alieh's scope, but we
should be clear about exactly what we are doing.

All errors that fall into ApiException seems too broad to me.

Justine

On Tue, Jul 2, 2024 at 10:51 AM Alieh Saeedi 
wrote:

> Hey Chris,
> thanks for sharing your concerns.
>
> 1) About the language of KIP (or maybe later in Javadocs): Is that alright
> if I write all errors that fall into the `ApiException` category thrown
> (actually returned) by Producer?
> 2) About future expansion: do you have any better suggestions for enum
> names? Do you think `IGNORE_API_EXEPTIONS` or something like that is a
> "better/more accurate" one?
>
> Bests,
> Alieh
>
> On Tue, Jul 2, 2024 at 7:29 PM Chris Egerton 
> wrote:
>
> > Hi Alieh and Justine,
> >
> > I'm concerned that we're settling on a definition of "poison pill" that's
> > easiest to tackle right now but may lead to shortcomings down the road. I
> > understand the relationship between this KIP and KAFKA-9279, and I can
> > totally get behind the desire to keep things small, focused, and simple
> in
> > the name of avoiding bugs. However, what I don't think is clear at all is
> > what the "specific circumstances" are that Justine mentioned. I had a
> > drastically different idea of what the intended behavioral change would
> be
> > before looking at the draft PR.
> >
> > I would like 1) for us to be clearer about the categories of errors that
> we
> > want to cover with this new API (especially since we'll have to find a
> > clear, succinct way to document this for users), and 2) to make sure that
> > if we do try to expand this API in the future, that we won't be painted
> > into a corner.
> >
> > For item 1, hopefully we can agree that the language in the KIP
> > for IGNORE_SEND_ERRORS ("The records causing irrecoverable errors are
> > excluded from the batch and the transaction is committed successfully.")
> is
> > pretty vague. If we start using the phrase "poison pill record" that
> could
> > help, but IMO more detail would still be needed. We know that we want to
> > include records that are so large that they can be immediately rejected
> by
> > the producer. But there are other cases that users might expect to be
> > handled. Does a record qualify as a poison pill if it targets a topic
> that
> > doesn't exist? Or if it targets a topic that the producer principal lacks
> > ACLs for? What if it fails broker-side validation (e.g., has a null key
> for
> > a compacted topic)?
> >
> > For item 2, this really depends on how narrow the scope of what we're
> doing
> > right now is. If we only handle a subset of the examples I laid out above
> > that could possibly be considered poison pills with this KIP, do we want
> to
> > lock ourselves in to never addressing more in the future, or can we
> choose
> > an API (probably just enum names would be the only important decision
> here)
> > that leaves room for more later?
> >
> > Best,
> >
> > Chris
> >
> >
> >
> > On Tue, Jul 2, 2024 at 12:28 PM Justine Olshan
> > 
> > wrote:
> >
> > > Chris and Alieh,
> > >
> > > My understanding is that this KIP is really only trying to solve an
> issue
> > > of a "poison pill" record that fails send().
> > > We've talked a lot about having a generic framework for all errors,
> but I
> > > don't think that is what this KIP is trying to do. Essentially the
> > request
> > > is to undo the change from KAFKA-9279
> > >  but under specific
> > > circumstances that are controlled. I really am concerned about opening
> > new
> > > avenues for bugs with EOS and hesitate to handle any other types of
> > errors.
> > > I think if we all agree on the problem that we are trying to solve, it
> is
> > > easier to agree on solutions.
> > >
> > > Justine
> > >
> > > On Mon, Jul 1, 2024 at 2:20 AM Alieh Saeedi
>  > >
> > > wrote:
> > >
> > > > Hi Matthias,
> > > > Thanks for the valid points you mentioned. I updated the KIP and the
> PR
> > > > with:
> > > > 1) mentioning that the new overloaded `send` throws
> > > `IllegalStateException`
> > > > if the user tries to ignore `send()` errors outside of a transaction.
> > > > 2) the default implementation in

Re: [DISCUSS] KIP-1059: Enable the Producer flush() method to clear the latest send() error

2024-07-02 Thread Alieh Saeedi
Hey Chris,
thanks for sharing your concerns.

1) About the language of KIP (or maybe later in Javadocs): Is that alright
if I write all errors that fall into the `ApiException` category thrown
(actually returned) by Producer?
2) About future expansion: do you have any better suggestions for enum
names? Do you think `IGNORE_API_EXEPTIONS` or something like that is a
"better/more accurate" one?

Bests,
Alieh

On Tue, Jul 2, 2024 at 7:29 PM Chris Egerton 
wrote:

> Hi Alieh and Justine,
>
> I'm concerned that we're settling on a definition of "poison pill" that's
> easiest to tackle right now but may lead to shortcomings down the road. I
> understand the relationship between this KIP and KAFKA-9279, and I can
> totally get behind the desire to keep things small, focused, and simple in
> the name of avoiding bugs. However, what I don't think is clear at all is
> what the "specific circumstances" are that Justine mentioned. I had a
> drastically different idea of what the intended behavioral change would be
> before looking at the draft PR.
>
> I would like 1) for us to be clearer about the categories of errors that we
> want to cover with this new API (especially since we'll have to find a
> clear, succinct way to document this for users), and 2) to make sure that
> if we do try to expand this API in the future, that we won't be painted
> into a corner.
>
> For item 1, hopefully we can agree that the language in the KIP
> for IGNORE_SEND_ERRORS ("The records causing irrecoverable errors are
> excluded from the batch and the transaction is committed successfully.") is
> pretty vague. If we start using the phrase "poison pill record" that could
> help, but IMO more detail would still be needed. We know that we want to
> include records that are so large that they can be immediately rejected by
> the producer. But there are other cases that users might expect to be
> handled. Does a record qualify as a poison pill if it targets a topic that
> doesn't exist? Or if it targets a topic that the producer principal lacks
> ACLs for? What if it fails broker-side validation (e.g., has a null key for
> a compacted topic)?
>
> For item 2, this really depends on how narrow the scope of what we're doing
> right now is. If we only handle a subset of the examples I laid out above
> that could possibly be considered poison pills with this KIP, do we want to
> lock ourselves in to never addressing more in the future, or can we choose
> an API (probably just enum names would be the only important decision here)
> that leaves room for more later?
>
> Best,
>
> Chris
>
>
>
> On Tue, Jul 2, 2024 at 12:28 PM Justine Olshan
> 
> wrote:
>
> > Chris and Alieh,
> >
> > My understanding is that this KIP is really only trying to solve an issue
> > of a "poison pill" record that fails send().
> > We've talked a lot about having a generic framework for all errors, but I
> > don't think that is what this KIP is trying to do. Essentially the
> request
> > is to undo the change from KAFKA-9279
> >  but under specific
> > circumstances that are controlled. I really am concerned about opening
> new
> > avenues for bugs with EOS and hesitate to handle any other types of
> errors.
> > I think if we all agree on the problem that we are trying to solve, it is
> > easier to agree on solutions.
> >
> > Justine
> >
> > On Mon, Jul 1, 2024 at 2:20 AM Alieh Saeedi  >
> > wrote:
> >
> > > Hi Matthias,
> > > Thanks for the valid points you mentioned. I updated the KIP and the PR
> > > with:
> > > 1) mentioning that the new overloaded `send` throws
> > `IllegalStateException`
> > > if the user tries to ignore `send()` errors outside of a transaction.
> > > 2) the default implementation in `Producer` interface throws an
> > > `UnsupportedOperationException`
> > >
> > > Hi Chris,
> > > Thanks for the feedback. I tried to clarify the points you listed:
> > > ---> we've narrowed the scope from any error that might take place
> > with
> > > producing a record to Kafka, to only the ones that are thrown directly
> > from
> > > Producer::send;
> > >
> > > From the very beginning and even since KIP-1038, the main purpose was
> to
> > > have "more flexibility," or, in other words, "giving the user the
> > > authority" to handle some specific exceptions thrown from the
> `Producer`.
> > > Due to the specific cases we had in mind, KIP-1038 was discarded and we
> > > decided to not define a `CustomExceptionHandler` for `Producer` and
> > instead
> > > treat the `send` failures in a different way. The main issue is that
> > `send`
> > > makes a transition to error state, which is undoable. In fact, one
> single
> > > poison pill record makes the whole batch fail. The former suggestions
> > that
> > > you agreed with have been all about un-doing this transition in `flush`
> > or
> > > `commit`. The new suggestion is to un-do (or better, NOT do) in `send`
> > due
> > > to the reasons listed in the discussions above.
> > 

Re: [DISCUSS] KIP-1059: Enable the Producer flush() method to clear the latest send() error

2024-07-02 Thread Chris Egerton
Hi Alieh and Justine,

I'm concerned that we're settling on a definition of "poison pill" that's
easiest to tackle right now but may lead to shortcomings down the road. I
understand the relationship between this KIP and KAFKA-9279, and I can
totally get behind the desire to keep things small, focused, and simple in
the name of avoiding bugs. However, what I don't think is clear at all is
what the "specific circumstances" are that Justine mentioned. I had a
drastically different idea of what the intended behavioral change would be
before looking at the draft PR.

I would like 1) for us to be clearer about the categories of errors that we
want to cover with this new API (especially since we'll have to find a
clear, succinct way to document this for users), and 2) to make sure that
if we do try to expand this API in the future, that we won't be painted
into a corner.

For item 1, hopefully we can agree that the language in the KIP
for IGNORE_SEND_ERRORS ("The records causing irrecoverable errors are
excluded from the batch and the transaction is committed successfully.") is
pretty vague. If we start using the phrase "poison pill record" that could
help, but IMO more detail would still be needed. We know that we want to
include records that are so large that they can be immediately rejected by
the producer. But there are other cases that users might expect to be
handled. Does a record qualify as a poison pill if it targets a topic that
doesn't exist? Or if it targets a topic that the producer principal lacks
ACLs for? What if it fails broker-side validation (e.g., has a null key for
a compacted topic)?

For item 2, this really depends on how narrow the scope of what we're doing
right now is. If we only handle a subset of the examples I laid out above
that could possibly be considered poison pills with this KIP, do we want to
lock ourselves in to never addressing more in the future, or can we choose
an API (probably just enum names would be the only important decision here)
that leaves room for more later?

Best,

Chris



On Tue, Jul 2, 2024 at 12:28 PM Justine Olshan 
wrote:

> Chris and Alieh,
>
> My understanding is that this KIP is really only trying to solve an issue
> of a "poison pill" record that fails send().
> We've talked a lot about having a generic framework for all errors, but I
> don't think that is what this KIP is trying to do. Essentially the request
> is to undo the change from KAFKA-9279
>  but under specific
> circumstances that are controlled. I really am concerned about opening new
> avenues for bugs with EOS and hesitate to handle any other types of errors.
> I think if we all agree on the problem that we are trying to solve, it is
> easier to agree on solutions.
>
> Justine
>
> On Mon, Jul 1, 2024 at 2:20 AM Alieh Saeedi 
> wrote:
>
> > Hi Matthias,
> > Thanks for the valid points you mentioned. I updated the KIP and the PR
> > with:
> > 1) mentioning that the new overloaded `send` throws
> `IllegalStateException`
> > if the user tries to ignore `send()` errors outside of a transaction.
> > 2) the default implementation in `Producer` interface throws an
> > `UnsupportedOperationException`
> >
> > Hi Chris,
> > Thanks for the feedback. I tried to clarify the points you listed:
> > ---> we've narrowed the scope from any error that might take place
> with
> > producing a record to Kafka, to only the ones that are thrown directly
> from
> > Producer::send;
> >
> > From the very beginning and even since KIP-1038, the main purpose was to
> > have "more flexibility," or, in other words, "giving the user the
> > authority" to handle some specific exceptions thrown from the `Producer`.
> > Due to the specific cases we had in mind, KIP-1038 was discarded and we
> > decided to not define a `CustomExceptionHandler` for `Producer` and
> instead
> > treat the `send` failures in a different way. The main issue is that
> `send`
> > makes a transition to error state, which is undoable. In fact, one single
> > poison pill record makes the whole batch fail. The former suggestions
> that
> > you agreed with have been all about un-doing this transition in `flush`
> or
> > `commit`. The new suggestion is to un-do (or better, NOT do) in `send`
> due
> > to the reasons listed in the discussions above.
> > Moreover, I would say that having such a large scope as you mentioned is
> > impossible. In the best case, we may have control over the `Producer`.
> What
> > shall we do with the broker? The `any error that might take place with
> > producing a record to Kafka` is too much, I think.
> >
> > ---> is this all we want to handle, and will it prevent us from
> > handling more in the future in an intuitive way?
> >
> > I think yes. This is all we want. Other sorts of errors such as having
> > problem with partition addition, producer fenced exception, etc seem to
> be
> > more serious issues. The intention was to handle problems created by
> > (maybe) a si

Re: [DISCUSS] KIP-1059: Enable the Producer flush() method to clear the latest send() error

2024-07-02 Thread Justine Olshan
Chris and Alieh,

My understanding is that this KIP is really only trying to solve an issue
of a "poison pill" record that fails send().
We've talked a lot about having a generic framework for all errors, but I
don't think that is what this KIP is trying to do. Essentially the request
is to undo the change from KAFKA-9279
 but under specific
circumstances that are controlled. I really am concerned about opening new
avenues for bugs with EOS and hesitate to handle any other types of errors.
I think if we all agree on the problem that we are trying to solve, it is
easier to agree on solutions.

Justine

On Mon, Jul 1, 2024 at 2:20 AM Alieh Saeedi 
wrote:

> Hi Matthias,
> Thanks for the valid points you mentioned. I updated the KIP and the PR
> with:
> 1) mentioning that the new overloaded `send` throws `IllegalStateException`
> if the user tries to ignore `send()` errors outside of a transaction.
> 2) the default implementation in `Producer` interface throws an
> `UnsupportedOperationException`
>
> Hi Chris,
> Thanks for the feedback. I tried to clarify the points you listed:
> ---> we've narrowed the scope from any error that might take place with
> producing a record to Kafka, to only the ones that are thrown directly from
> Producer::send;
>
> From the very beginning and even since KIP-1038, the main purpose was to
> have "more flexibility," or, in other words, "giving the user the
> authority" to handle some specific exceptions thrown from the `Producer`.
> Due to the specific cases we had in mind, KIP-1038 was discarded and we
> decided to not define a `CustomExceptionHandler` for `Producer` and instead
> treat the `send` failures in a different way. The main issue is that `send`
> makes a transition to error state, which is undoable. In fact, one single
> poison pill record makes the whole batch fail. The former suggestions that
> you agreed with have been all about un-doing this transition in `flush` or
> `commit`. The new suggestion is to un-do (or better, NOT do) in `send` due
> to the reasons listed in the discussions above.
> Moreover, I would say that having such a large scope as you mentioned is
> impossible. In the best case, we may have control over the `Producer`. What
> shall we do with the broker? The `any error that might take place with
> producing a record to Kafka` is too much, I think.
>
> ---> is this all we want to handle, and will it prevent us from
> handling more in the future in an intuitive way?
>
> I think yes. This is all we want. Other sorts of errors such as having
> problem with partition addition, producer fenced exception, etc seem to be
> more serious issues. The intention was to handle problems created by
> (maybe) a single poison pill record. BTW, I do not see any obstacles to
> future changes.
>
> Bests,
> Alieh
>
> On Sat, Jun 29, 2024 at 3:03 AM Chris Egerton 
> wrote:
>
> > Ah, sorry--spoke too soon. The PR doesn't show that errors thrown from
> > Producer::send are handled, but instead, ApiException instances that are
> > caught inside KafkaProducer::doSend and are handled by returning an
> > already-failed future are. I think the same question still applies (is
> this
> > all we want to handle, and will it prevent us from handling more in the
> > future in an intuitive way), though.
> >
> > On Fri, Jun 28, 2024 at 8:57 PM Chris Egerton  wrote:
> >
> > > Hi Alieh,
> > >
> > > This KIP has evolved a lot since I last looked at it, but the changes
> > seem
> > > well thought-out both in semantics and API. One clarifying question I
> > have
> > > is that it looks based on the draft PR that we've narrowed the scope
> from
> > > any error that might take place with producing a record to Kafka, to
> only
> > > the ones that are thrown directly from Producer::send; is that the
> > intended
> > > behavior here? And if so, do you have thoughts on how we might design a
> > > follow-up KIP that would catch all errors (including ones reported
> > > asynchronously instead of synchronously)? I'd like it if we could leave
> > the
> > > door open for that without painting ourselves into too much of a corner
> > > with the API design for this KIP.
> > >
> > > Cheers,
> > >
> > > Chris
> > >
> > > On Fri, Jun 28, 2024 at 6:31 PM Matthias J. Sax 
> > wrote:
> > >
> > >> Thanks Alieh,
> > >>
> > >> it seems this KIP can just pick between a couple of tradeoffs. Adding
> an
> > >> overloaded `send()` as the KIP propose makes sense to me and seems to
> > >> provides the cleanest solution compare to there options we discussed.
> > >>
> > >> Given the explicit name of the passed-in option that highlights that
> the
> > >> option is for TX only make is pretty clear and avoids the issue of
> > >> `flush()` ambiguity.
> > >>
> > >>
> > >> Nit: We should make clear on the KIP though, that the new `send()`
> > >> overload would throw an `IllegalStateException` if TX are not used
> > >> (similar to other TX methods like initTx(), etc)
> 

Re: [DISCUSS] KIP-1059: Enable the Producer flush() method to clear the latest send() error

2024-07-01 Thread Alieh Saeedi
Hi Matthias,
Thanks for the valid points you mentioned. I updated the KIP and the PR
with:
1) mentioning that the new overloaded `send` throws `IllegalStateException`
if the user tries to ignore `send()` errors outside of a transaction.
2) the default implementation in `Producer` interface throws an
`UnsupportedOperationException`

Hi Chris,
Thanks for the feedback. I tried to clarify the points you listed:
---> we've narrowed the scope from any error that might take place with
producing a record to Kafka, to only the ones that are thrown directly from
Producer::send;

>From the very beginning and even since KIP-1038, the main purpose was to
have "more flexibility," or, in other words, "giving the user the
authority" to handle some specific exceptions thrown from the `Producer`.
Due to the specific cases we had in mind, KIP-1038 was discarded and we
decided to not define a `CustomExceptionHandler` for `Producer` and instead
treat the `send` failures in a different way. The main issue is that `send`
makes a transition to error state, which is undoable. In fact, one single
poison pill record makes the whole batch fail. The former suggestions that
you agreed with have been all about un-doing this transition in `flush` or
`commit`. The new suggestion is to un-do (or better, NOT do) in `send` due
to the reasons listed in the discussions above.
Moreover, I would say that having such a large scope as you mentioned is
impossible. In the best case, we may have control over the `Producer`. What
shall we do with the broker? The `any error that might take place with
producing a record to Kafka` is too much, I think.

---> is this all we want to handle, and will it prevent us from
handling more in the future in an intuitive way?

I think yes. This is all we want. Other sorts of errors such as having
problem with partition addition, producer fenced exception, etc seem to be
more serious issues. The intention was to handle problems created by
(maybe) a single poison pill record. BTW, I do not see any obstacles to
future changes.

Bests,
Alieh

On Sat, Jun 29, 2024 at 3:03 AM Chris Egerton 
wrote:

> Ah, sorry--spoke too soon. The PR doesn't show that errors thrown from
> Producer::send are handled, but instead, ApiException instances that are
> caught inside KafkaProducer::doSend and are handled by returning an
> already-failed future are. I think the same question still applies (is this
> all we want to handle, and will it prevent us from handling more in the
> future in an intuitive way), though.
>
> On Fri, Jun 28, 2024 at 8:57 PM Chris Egerton  wrote:
>
> > Hi Alieh,
> >
> > This KIP has evolved a lot since I last looked at it, but the changes
> seem
> > well thought-out both in semantics and API. One clarifying question I
> have
> > is that it looks based on the draft PR that we've narrowed the scope from
> > any error that might take place with producing a record to Kafka, to only
> > the ones that are thrown directly from Producer::send; is that the
> intended
> > behavior here? And if so, do you have thoughts on how we might design a
> > follow-up KIP that would catch all errors (including ones reported
> > asynchronously instead of synchronously)? I'd like it if we could leave
> the
> > door open for that without painting ourselves into too much of a corner
> > with the API design for this KIP.
> >
> > Cheers,
> >
> > Chris
> >
> > On Fri, Jun 28, 2024 at 6:31 PM Matthias J. Sax 
> wrote:
> >
> >> Thanks Alieh,
> >>
> >> it seems this KIP can just pick between a couple of tradeoffs. Adding an
> >> overloaded `send()` as the KIP propose makes sense to me and seems to
> >> provides the cleanest solution compare to there options we discussed.
> >>
> >> Given the explicit name of the passed-in option that highlights that the
> >> option is for TX only make is pretty clear and avoids the issue of
> >> `flush()` ambiguity.
> >>
> >>
> >> Nit: We should make clear on the KIP though, that the new `send()`
> >> overload would throw an `IllegalStateException` if TX are not used
> >> (similar to other TX methods like initTx(), etc)
> >>
> >>
> >> About the `Producer` interface, I am not sure how this was done in the
> >> past (eg, KIP-266 added `Consumer.poll(Duration)` w/o a default
> >> implementation), if we need a default implementation for backward
> >> compatibility or not? If we do want to add one, I think it would be
> >> appropriate to throw an `UnsupportedOperationException` by default,
> >> instead of just keeping the default impl empty?
> >>
> >>
> >> My points are rather minor, and should not block this KIP though.
> >> Overall LGTM.
> >>
> >>
> >>
> >> -Matthias
> >>
> >> On 6/27/24 1:28 PM, Alieh Saeedi wrote:
> >> > Hi Justine,
> >> >
> >> > Thanks for the suggestion.
> >> > Making applications to validate every single record is not the best
> way,
> >> > from an efficiency point of view.
> >> > Moreover, between changing the behavior of the Producer in `send` and
> >> > `commitTnx`, the former seem

Re: [DISCUSS] KIP-1059: Enable the Producer flush() method to clear the latest send() error

2024-06-28 Thread Chris Egerton
Ah, sorry--spoke too soon. The PR doesn't show that errors thrown from
Producer::send are handled, but instead, ApiException instances that are
caught inside KafkaProducer::doSend and are handled by returning an
already-failed future are. I think the same question still applies (is this
all we want to handle, and will it prevent us from handling more in the
future in an intuitive way), though.

On Fri, Jun 28, 2024 at 8:57 PM Chris Egerton  wrote:

> Hi Alieh,
>
> This KIP has evolved a lot since I last looked at it, but the changes seem
> well thought-out both in semantics and API. One clarifying question I have
> is that it looks based on the draft PR that we've narrowed the scope from
> any error that might take place with producing a record to Kafka, to only
> the ones that are thrown directly from Producer::send; is that the intended
> behavior here? And if so, do you have thoughts on how we might design a
> follow-up KIP that would catch all errors (including ones reported
> asynchronously instead of synchronously)? I'd like it if we could leave the
> door open for that without painting ourselves into too much of a corner
> with the API design for this KIP.
>
> Cheers,
>
> Chris
>
> On Fri, Jun 28, 2024 at 6:31 PM Matthias J. Sax  wrote:
>
>> Thanks Alieh,
>>
>> it seems this KIP can just pick between a couple of tradeoffs. Adding an
>> overloaded `send()` as the KIP propose makes sense to me and seems to
>> provides the cleanest solution compare to there options we discussed.
>>
>> Given the explicit name of the passed-in option that highlights that the
>> option is for TX only make is pretty clear and avoids the issue of
>> `flush()` ambiguity.
>>
>>
>> Nit: We should make clear on the KIP though, that the new `send()`
>> overload would throw an `IllegalStateException` if TX are not used
>> (similar to other TX methods like initTx(), etc)
>>
>>
>> About the `Producer` interface, I am not sure how this was done in the
>> past (eg, KIP-266 added `Consumer.poll(Duration)` w/o a default
>> implementation), if we need a default implementation for backward
>> compatibility or not? If we do want to add one, I think it would be
>> appropriate to throw an `UnsupportedOperationException` by default,
>> instead of just keeping the default impl empty?
>>
>>
>> My points are rather minor, and should not block this KIP though.
>> Overall LGTM.
>>
>>
>>
>> -Matthias
>>
>> On 6/27/24 1:28 PM, Alieh Saeedi wrote:
>> > Hi Justine,
>> >
>> > Thanks for the suggestion.
>> > Making applications to validate every single record is not the best way,
>> > from an efficiency point of view.
>> > Moreover, between changing the behavior of the Producer in `send` and
>> > `commitTnx`, the former seems more reasonable and clean.
>> >
>> > Bests,
>> > Alieh
>> >
>> > On Thu, Jun 27, 2024 at 8:14 PM Justine Olshan
>> 
>> > wrote:
>> >
>> >> Hey Alieh,
>> >>
>> >> I see there are two options now. So folks will be discussing the
>> approaches
>> >> and deciding the best way forward before we vote?
>> >> I do think there could be a problem with the approach on commit if we
>> get
>> >> stuck on an earlier error and have more records (potentially on new
>> >> partitions) to commit as the current PR is implemented.
>> >>
>> >> I guess this takes us back to the question of whether the error should
>> be
>> >> cleared on send.
>> >>
>> >> (And I guess at the back of my mind, I'm wondering if there is a way
>> we can
>> >> validate the "posion pill" records application side before we even try
>> to
>> >> send them)
>> >>
>> >> Justine
>> >>
>> >> On Wed, Jun 26, 2024 at 4:38 PM Alieh Saeedi
>> > >>>
>> >> wrote:
>> >>
>> >>> Hi Justine,
>> >>>
>> >>> I did not update the KIP with `TxnSendOption` since I thought it'd be
>> >>> better discussed here beforehand.
>> >>> right now, there are 2 PRs:
>> >>> - the PR that implements the current version of the KIP:
>> >>> https://github.com/apache/kafka/pull/16332
>> >>> - the POC PR that clarifies the `TxnSendOption`:
>> >>> https://github.com/apache/kafka/pull/16465
>> >>>
>> >>> Bests,
>> >>> Alieh
>> >>>
>> >>> On Thu, Jun 27, 2024 at 12:42 AM Justine Olshan
>> >>>  wrote:
>> >>>
>>  Hey Alieh,
>> 
>>  I think I am a little confused. Are the 3 points above addressed by
>> the
>> >>> KIP
>>  or did something change? The PR seems to not include this change and
>> >>> still
>>  has the CommitOption as well.
>> 
>>  Thanks,
>>  Justine
>> 
>>  On Wed, Jun 26, 2024 at 2:15 PM Alieh Saeedi
>> >>> > >
>>  wrote:
>> 
>> > Hi all,
>> >
>> >
>> > Looking at the PR 
>> > corresponding to the KIP, there are some points worthy of mention:
>> >
>> >
>> > 1) clearing the error ends up dirty/messy code in
>> >> `TransactionManager`.
>> >
>> > 2) By clearing the error, we are actually doing an illegal
>> transition
>>  from
>> > `ABORTABLE_ERROR` to `IN_TRANSACTION` wh

Re: [DISCUSS] KIP-1059: Enable the Producer flush() method to clear the latest send() error

2024-06-28 Thread Chris Egerton
Hi Alieh,

This KIP has evolved a lot since I last looked at it, but the changes seem
well thought-out both in semantics and API. One clarifying question I have
is that it looks based on the draft PR that we've narrowed the scope from
any error that might take place with producing a record to Kafka, to only
the ones that are thrown directly from Producer::send; is that the intended
behavior here? And if so, do you have thoughts on how we might design a
follow-up KIP that would catch all errors (including ones reported
asynchronously instead of synchronously)? I'd like it if we could leave the
door open for that without painting ourselves into too much of a corner
with the API design for this KIP.

Cheers,

Chris

On Fri, Jun 28, 2024 at 6:31 PM Matthias J. Sax  wrote:

> Thanks Alieh,
>
> it seems this KIP can just pick between a couple of tradeoffs. Adding an
> overloaded `send()` as the KIP propose makes sense to me and seems to
> provides the cleanest solution compare to there options we discussed.
>
> Given the explicit name of the passed-in option that highlights that the
> option is for TX only make is pretty clear and avoids the issue of
> `flush()` ambiguity.
>
>
> Nit: We should make clear on the KIP though, that the new `send()`
> overload would throw an `IllegalStateException` if TX are not used
> (similar to other TX methods like initTx(), etc)
>
>
> About the `Producer` interface, I am not sure how this was done in the
> past (eg, KIP-266 added `Consumer.poll(Duration)` w/o a default
> implementation), if we need a default implementation for backward
> compatibility or not? If we do want to add one, I think it would be
> appropriate to throw an `UnsupportedOperationException` by default,
> instead of just keeping the default impl empty?
>
>
> My points are rather minor, and should not block this KIP though.
> Overall LGTM.
>
>
>
> -Matthias
>
> On 6/27/24 1:28 PM, Alieh Saeedi wrote:
> > Hi Justine,
> >
> > Thanks for the suggestion.
> > Making applications to validate every single record is not the best way,
> > from an efficiency point of view.
> > Moreover, between changing the behavior of the Producer in `send` and
> > `commitTnx`, the former seems more reasonable and clean.
> >
> > Bests,
> > Alieh
> >
> > On Thu, Jun 27, 2024 at 8:14 PM Justine Olshan
> 
> > wrote:
> >
> >> Hey Alieh,
> >>
> >> I see there are two options now. So folks will be discussing the
> approaches
> >> and deciding the best way forward before we vote?
> >> I do think there could be a problem with the approach on commit if we
> get
> >> stuck on an earlier error and have more records (potentially on new
> >> partitions) to commit as the current PR is implemented.
> >>
> >> I guess this takes us back to the question of whether the error should
> be
> >> cleared on send.
> >>
> >> (And I guess at the back of my mind, I'm wondering if there is a way we
> can
> >> validate the "posion pill" records application side before we even try
> to
> >> send them)
> >>
> >> Justine
> >>
> >> On Wed, Jun 26, 2024 at 4:38 PM Alieh Saeedi
>  >>>
> >> wrote:
> >>
> >>> Hi Justine,
> >>>
> >>> I did not update the KIP with `TxnSendOption` since I thought it'd be
> >>> better discussed here beforehand.
> >>> right now, there are 2 PRs:
> >>> - the PR that implements the current version of the KIP:
> >>> https://github.com/apache/kafka/pull/16332
> >>> - the POC PR that clarifies the `TxnSendOption`:
> >>> https://github.com/apache/kafka/pull/16465
> >>>
> >>> Bests,
> >>> Alieh
> >>>
> >>> On Thu, Jun 27, 2024 at 12:42 AM Justine Olshan
> >>>  wrote:
> >>>
>  Hey Alieh,
> 
>  I think I am a little confused. Are the 3 points above addressed by
> the
> >>> KIP
>  or did something change? The PR seems to not include this change and
> >>> still
>  has the CommitOption as well.
> 
>  Thanks,
>  Justine
> 
>  On Wed, Jun 26, 2024 at 2:15 PM Alieh Saeedi
> >>>  >
>  wrote:
> 
> > Hi all,
> >
> >
> > Looking at the PR 
> > corresponding to the KIP, there are some points worthy of mention:
> >
> >
> > 1) clearing the error ends up dirty/messy code in
> >> `TransactionManager`.
> >
> > 2) By clearing the error, we are actually doing an illegal transition
>  from
> > `ABORTABLE_ERROR` to `IN_TRANSACTION` which is conceptually not
>  acceptable.
> > This can be the root cause of some issues, with perhaps further
> >> future
> > changes by others.
> >
> > 3) If the poison pill record `r1` causes a transition to the error
> >>> state
> > and then the next record `r2` requires adding a partition to the
> > transaction, the action fails due to being in the error state. In
> >> this
> > case, clearing errors during `commitTnx(CLEAR_SEND_ERROR)` is too
> >> late.
> > However, this case can NOT be the main concern as soon as KIP-890 is
>  fully
> > implemented.
> >

Re: [DISCUSS] KIP-1059: Enable the Producer flush() method to clear the latest send() error

2024-06-28 Thread Matthias J. Sax

Thanks Alieh,

it seems this KIP can just pick between a couple of tradeoffs. Adding an 
overloaded `send()` as the KIP propose makes sense to me and seems to 
provides the cleanest solution compare to there options we discussed.


Given the explicit name of the passed-in option that highlights that the 
option is for TX only make is pretty clear and avoids the issue of 
`flush()` ambiguity.



Nit: We should make clear on the KIP though, that the new `send()` 
overload would throw an `IllegalStateException` if TX are not used 
(similar to other TX methods like initTx(), etc)



About the `Producer` interface, I am not sure how this was done in the 
past (eg, KIP-266 added `Consumer.poll(Duration)` w/o a default 
implementation), if we need a default implementation for backward 
compatibility or not? If we do want to add one, I think it would be 
appropriate to throw an `UnsupportedOperationException` by default, 
instead of just keeping the default impl empty?



My points are rather minor, and should not block this KIP though. 
Overall LGTM.




-Matthias

On 6/27/24 1:28 PM, Alieh Saeedi wrote:

Hi Justine,

Thanks for the suggestion.
Making applications to validate every single record is not the best way,
from an efficiency point of view.
Moreover, between changing the behavior of the Producer in `send` and
`commitTnx`, the former seems more reasonable and clean.

Bests,
Alieh

On Thu, Jun 27, 2024 at 8:14 PM Justine Olshan 
wrote:


Hey Alieh,

I see there are two options now. So folks will be discussing the approaches
and deciding the best way forward before we vote?
I do think there could be a problem with the approach on commit if we get
stuck on an earlier error and have more records (potentially on new
partitions) to commit as the current PR is implemented.

I guess this takes us back to the question of whether the error should be
cleared on send.

(And I guess at the back of my mind, I'm wondering if there is a way we can
validate the "posion pill" records application side before we even try to
send them)

Justine

On Wed, Jun 26, 2024 at 4:38 PM Alieh Saeedi 


wrote:


Hi Justine,

I did not update the KIP with `TxnSendOption` since I thought it'd be
better discussed here beforehand.
right now, there are 2 PRs:
- the PR that implements the current version of the KIP:
https://github.com/apache/kafka/pull/16332
- the POC PR that clarifies the `TxnSendOption`:
https://github.com/apache/kafka/pull/16465

Bests,
Alieh

On Thu, Jun 27, 2024 at 12:42 AM Justine Olshan
 wrote:


Hey Alieh,

I think I am a little confused. Are the 3 points above addressed by the

KIP

or did something change? The PR seems to not include this change and

still

has the CommitOption as well.

Thanks,
Justine

On Wed, Jun 26, 2024 at 2:15 PM Alieh Saeedi




wrote:


Hi all,


Looking at the PR 
corresponding to the KIP, there are some points worthy of mention:


1) clearing the error ends up dirty/messy code in

`TransactionManager`.


2) By clearing the error, we are actually doing an illegal transition

from

`ABORTABLE_ERROR` to `IN_TRANSACTION` which is conceptually not

acceptable.

This can be the root cause of some issues, with perhaps further

future

changes by others.

3) If the poison pill record `r1` causes a transition to the error

state

and then the next record `r2` requires adding a partition to the
transaction, the action fails due to being in the error state. In

this

case, clearing errors during `commitTnx(CLEAR_SEND_ERROR)` is too

late.

However, this case can NOT be the main concern as soon as KIP-890 is

fully

implemented.


My suggestion is to solve the problem where it arises. If the

transition

to

the error state does not happen during `send()`, we do not need to

clear

the error later. Therefore, instead of `CommitOption`, we can define

a

`TxnSendOption` and prevent the `send()` method from going to the

error

state in case 1) we're in a transaction and 2) the user asked for
IGONRE_SEND_ERRORS. For more clarity, you can take a look at the POC

PR

.

Cheers,
Alieh











Re: [DISCUSS] KIP-1059: Enable the Producer flush() method to clear the latest send() error

2024-06-27 Thread Alieh Saeedi
Hi Justine,

Thanks for the suggestion.
Making applications to validate every single record is not the best way,
from an efficiency point of view.
Moreover, between changing the behavior of the Producer in `send` and
`commitTnx`, the former seems more reasonable and clean.

Bests,
Alieh

On Thu, Jun 27, 2024 at 8:14 PM Justine Olshan 
wrote:

> Hey Alieh,
>
> I see there are two options now. So folks will be discussing the approaches
> and deciding the best way forward before we vote?
> I do think there could be a problem with the approach on commit if we get
> stuck on an earlier error and have more records (potentially on new
> partitions) to commit as the current PR is implemented.
>
> I guess this takes us back to the question of whether the error should be
> cleared on send.
>
> (And I guess at the back of my mind, I'm wondering if there is a way we can
> validate the "posion pill" records application side before we even try to
> send them)
>
> Justine
>
> On Wed, Jun 26, 2024 at 4:38 PM Alieh Saeedi  >
> wrote:
>
> > Hi Justine,
> >
> > I did not update the KIP with `TxnSendOption` since I thought it'd be
> > better discussed here beforehand.
> > right now, there are 2 PRs:
> > - the PR that implements the current version of the KIP:
> > https://github.com/apache/kafka/pull/16332
> > - the POC PR that clarifies the `TxnSendOption`:
> > https://github.com/apache/kafka/pull/16465
> >
> > Bests,
> > Alieh
> >
> > On Thu, Jun 27, 2024 at 12:42 AM Justine Olshan
> >  wrote:
> >
> > > Hey Alieh,
> > >
> > > I think I am a little confused. Are the 3 points above addressed by the
> > KIP
> > > or did something change? The PR seems to not include this change and
> > still
> > > has the CommitOption as well.
> > >
> > > Thanks,
> > > Justine
> > >
> > > On Wed, Jun 26, 2024 at 2:15 PM Alieh Saeedi
> >  > > >
> > > wrote:
> > >
> > > > Hi all,
> > > >
> > > >
> > > > Looking at the PR 
> > > > corresponding to the KIP, there are some points worthy of mention:
> > > >
> > > >
> > > > 1) clearing the error ends up dirty/messy code in
> `TransactionManager`.
> > > >
> > > > 2) By clearing the error, we are actually doing an illegal transition
> > > from
> > > > `ABORTABLE_ERROR` to `IN_TRANSACTION` which is conceptually not
> > > acceptable.
> > > > This can be the root cause of some issues, with perhaps further
> future
> > > > changes by others.
> > > >
> > > > 3) If the poison pill record `r1` causes a transition to the error
> > state
> > > > and then the next record `r2` requires adding a partition to the
> > > > transaction, the action fails due to being in the error state. In
> this
> > > > case, clearing errors during `commitTnx(CLEAR_SEND_ERROR)` is too
> late.
> > > > However, this case can NOT be the main concern as soon as KIP-890 is
> > > fully
> > > > implemented.
> > > >
> > > >
> > > > My suggestion is to solve the problem where it arises. If the
> > transition
> > > to
> > > > the error state does not happen during `send()`, we do not need to
> > clear
> > > > the error later. Therefore, instead of `CommitOption`, we can define
> a
> > > > `TxnSendOption` and prevent the `send()` method from going to the
> error
> > > > state in case 1) we're in a transaction and 2) the user asked for
> > > > IGONRE_SEND_ERRORS. For more clarity, you can take a look at the POC
> PR
> > > > .
> > > >
> > > > Cheers,
> > > > Alieh
> > > >
> > >
> >
>


Re: [DISCUSS] KIP-1059: Enable the Producer flush() method to clear the latest send() error

2024-06-27 Thread Alieh Saeedi
Hi all,

I updated the KIP as follows:
1) added `TxnSendOption` with two possible values (NONE,
IGNORE_SEND_ERRORS).
2) added the new `send` method with three input parameters to `Producer`
and `KafkaProducer`.
3) removed `CommitOption` and `commitTransaction(CommitOption)` from
`Producer` and `KafkaProducer`.

Thanks,
Alieh

On Thu, Jun 27, 2024 at 8:14 PM Justine Olshan 
wrote:

> Hey Alieh,
>
> I see there are two options now. So folks will be discussing the approaches
> and deciding the best way forward before we vote?
> I do think there could be a problem with the approach on commit if we get
> stuck on an earlier error and have more records (potentially on new
> partitions) to commit as the current PR is implemented.
>
> I guess this takes us back to the question of whether the error should be
> cleared on send.
>
> (And I guess at the back of my mind, I'm wondering if there is a way we can
> validate the "posion pill" records application side before we even try to
> send them)
>
> Justine
>
> On Wed, Jun 26, 2024 at 4:38 PM Alieh Saeedi  >
> wrote:
>
> > Hi Justine,
> >
> > I did not update the KIP with `TxnSendOption` since I thought it'd be
> > better discussed here beforehand.
> > right now, there are 2 PRs:
> > - the PR that implements the current version of the KIP:
> > https://github.com/apache/kafka/pull/16332
> > - the POC PR that clarifies the `TxnSendOption`:
> > https://github.com/apache/kafka/pull/16465
> >
> > Bests,
> > Alieh
> >
> > On Thu, Jun 27, 2024 at 12:42 AM Justine Olshan
> >  wrote:
> >
> > > Hey Alieh,
> > >
> > > I think I am a little confused. Are the 3 points above addressed by the
> > KIP
> > > or did something change? The PR seems to not include this change and
> > still
> > > has the CommitOption as well.
> > >
> > > Thanks,
> > > Justine
> > >
> > > On Wed, Jun 26, 2024 at 2:15 PM Alieh Saeedi
> >  > > >
> > > wrote:
> > >
> > > > Hi all,
> > > >
> > > >
> > > > Looking at the PR 
> > > > corresponding to the KIP, there are some points worthy of mention:
> > > >
> > > >
> > > > 1) clearing the error ends up dirty/messy code in
> `TransactionManager`.
> > > >
> > > > 2) By clearing the error, we are actually doing an illegal transition
> > > from
> > > > `ABORTABLE_ERROR` to `IN_TRANSACTION` which is conceptually not
> > > acceptable.
> > > > This can be the root cause of some issues, with perhaps further
> future
> > > > changes by others.
> > > >
> > > > 3) If the poison pill record `r1` causes a transition to the error
> > state
> > > > and then the next record `r2` requires adding a partition to the
> > > > transaction, the action fails due to being in the error state. In
> this
> > > > case, clearing errors during `commitTnx(CLEAR_SEND_ERROR)` is too
> late.
> > > > However, this case can NOT be the main concern as soon as KIP-890 is
> > > fully
> > > > implemented.
> > > >
> > > >
> > > > My suggestion is to solve the problem where it arises. If the
> > transition
> > > to
> > > > the error state does not happen during `send()`, we do not need to
> > clear
> > > > the error later. Therefore, instead of `CommitOption`, we can define
> a
> > > > `TxnSendOption` and prevent the `send()` method from going to the
> error
> > > > state in case 1) we're in a transaction and 2) the user asked for
> > > > IGONRE_SEND_ERRORS. For more clarity, you can take a look at the POC
> PR
> > > > .
> > > >
> > > > Cheers,
> > > > Alieh
> > > >
> > >
> >
>


Re: [DISCUSS] KIP-1059: Enable the Producer flush() method to clear the latest send() error

2024-06-27 Thread Justine Olshan
Hey Alieh,

I see there are two options now. So folks will be discussing the approaches
and deciding the best way forward before we vote?
I do think there could be a problem with the approach on commit if we get
stuck on an earlier error and have more records (potentially on new
partitions) to commit as the current PR is implemented.

I guess this takes us back to the question of whether the error should be
cleared on send.

(And I guess at the back of my mind, I'm wondering if there is a way we can
validate the "posion pill" records application side before we even try to
send them)

Justine

On Wed, Jun 26, 2024 at 4:38 PM Alieh Saeedi 
wrote:

> Hi Justine,
>
> I did not update the KIP with `TxnSendOption` since I thought it'd be
> better discussed here beforehand.
> right now, there are 2 PRs:
> - the PR that implements the current version of the KIP:
> https://github.com/apache/kafka/pull/16332
> - the POC PR that clarifies the `TxnSendOption`:
> https://github.com/apache/kafka/pull/16465
>
> Bests,
> Alieh
>
> On Thu, Jun 27, 2024 at 12:42 AM Justine Olshan
>  wrote:
>
> > Hey Alieh,
> >
> > I think I am a little confused. Are the 3 points above addressed by the
> KIP
> > or did something change? The PR seems to not include this change and
> still
> > has the CommitOption as well.
> >
> > Thanks,
> > Justine
> >
> > On Wed, Jun 26, 2024 at 2:15 PM Alieh Saeedi
>  > >
> > wrote:
> >
> > > Hi all,
> > >
> > >
> > > Looking at the PR 
> > > corresponding to the KIP, there are some points worthy of mention:
> > >
> > >
> > > 1) clearing the error ends up dirty/messy code in `TransactionManager`.
> > >
> > > 2) By clearing the error, we are actually doing an illegal transition
> > from
> > > `ABORTABLE_ERROR` to `IN_TRANSACTION` which is conceptually not
> > acceptable.
> > > This can be the root cause of some issues, with perhaps further future
> > > changes by others.
> > >
> > > 3) If the poison pill record `r1` causes a transition to the error
> state
> > > and then the next record `r2` requires adding a partition to the
> > > transaction, the action fails due to being in the error state. In this
> > > case, clearing errors during `commitTnx(CLEAR_SEND_ERROR)` is too late.
> > > However, this case can NOT be the main concern as soon as KIP-890 is
> > fully
> > > implemented.
> > >
> > >
> > > My suggestion is to solve the problem where it arises. If the
> transition
> > to
> > > the error state does not happen during `send()`, we do not need to
> clear
> > > the error later. Therefore, instead of `CommitOption`, we can define a
> > > `TxnSendOption` and prevent the `send()` method from going to the error
> > > state in case 1) we're in a transaction and 2) the user asked for
> > > IGONRE_SEND_ERRORS. For more clarity, you can take a look at the POC PR
> > > .
> > >
> > > Cheers,
> > > Alieh
> > >
> >
>


Re: [DISCUSS] KIP-1059: Enable the Producer flush() method to clear the latest send() error

2024-06-26 Thread Alieh Saeedi
Hi Justine,

I did not update the KIP with `TxnSendOption` since I thought it'd be
better discussed here beforehand.
right now, there are 2 PRs:
- the PR that implements the current version of the KIP:
https://github.com/apache/kafka/pull/16332
- the POC PR that clarifies the `TxnSendOption`:
https://github.com/apache/kafka/pull/16465

Bests,
Alieh

On Thu, Jun 27, 2024 at 12:42 AM Justine Olshan
 wrote:

> Hey Alieh,
>
> I think I am a little confused. Are the 3 points above addressed by the KIP
> or did something change? The PR seems to not include this change and still
> has the CommitOption as well.
>
> Thanks,
> Justine
>
> On Wed, Jun 26, 2024 at 2:15 PM Alieh Saeedi  >
> wrote:
>
> > Hi all,
> >
> >
> > Looking at the PR 
> > corresponding to the KIP, there are some points worthy of mention:
> >
> >
> > 1) clearing the error ends up dirty/messy code in `TransactionManager`.
> >
> > 2) By clearing the error, we are actually doing an illegal transition
> from
> > `ABORTABLE_ERROR` to `IN_TRANSACTION` which is conceptually not
> acceptable.
> > This can be the root cause of some issues, with perhaps further future
> > changes by others.
> >
> > 3) If the poison pill record `r1` causes a transition to the error state
> > and then the next record `r2` requires adding a partition to the
> > transaction, the action fails due to being in the error state. In this
> > case, clearing errors during `commitTnx(CLEAR_SEND_ERROR)` is too late.
> > However, this case can NOT be the main concern as soon as KIP-890 is
> fully
> > implemented.
> >
> >
> > My suggestion is to solve the problem where it arises. If the transition
> to
> > the error state does not happen during `send()`, we do not need to clear
> > the error later. Therefore, instead of `CommitOption`, we can define a
> > `TxnSendOption` and prevent the `send()` method from going to the error
> > state in case 1) we're in a transaction and 2) the user asked for
> > IGONRE_SEND_ERRORS. For more clarity, you can take a look at the POC PR
> > .
> >
> > Cheers,
> > Alieh
> >
>


Re: [DISCUSS] KIP-1059: Enable the Producer flush() method to clear the latest send() error

2024-06-26 Thread Justine Olshan
Hey Alieh,

I think I am a little confused. Are the 3 points above addressed by the KIP
or did something change? The PR seems to not include this change and still
has the CommitOption as well.

Thanks,
Justine

On Wed, Jun 26, 2024 at 2:15 PM Alieh Saeedi 
wrote:

> Hi all,
>
>
> Looking at the PR 
> corresponding to the KIP, there are some points worthy of mention:
>
>
> 1) clearing the error ends up dirty/messy code in `TransactionManager`.
>
> 2) By clearing the error, we are actually doing an illegal transition from
> `ABORTABLE_ERROR` to `IN_TRANSACTION` which is conceptually not acceptable.
> This can be the root cause of some issues, with perhaps further future
> changes by others.
>
> 3) If the poison pill record `r1` causes a transition to the error state
> and then the next record `r2` requires adding a partition to the
> transaction, the action fails due to being in the error state. In this
> case, clearing errors during `commitTnx(CLEAR_SEND_ERROR)` is too late.
> However, this case can NOT be the main concern as soon as KIP-890 is fully
> implemented.
>
>
> My suggestion is to solve the problem where it arises. If the transition to
> the error state does not happen during `send()`, we do not need to clear
> the error later. Therefore, instead of `CommitOption`, we can define a
> `TxnSendOption` and prevent the `send()` method from going to the error
> state in case 1) we're in a transaction and 2) the user asked for
> IGONRE_SEND_ERRORS. For more clarity, you can take a look at the POC PR
> .
>
> Cheers,
> Alieh
>


Re: [DISCUSS] KIP-1059: Enable the Producer flush() method to clear the latest send() error

2024-06-26 Thread Alieh Saeedi
Hi all,


Looking at the PR 
corresponding to the KIP, there are some points worthy of mention:


1) clearing the error ends up dirty/messy code in `TransactionManager`.

2) By clearing the error, we are actually doing an illegal transition from
`ABORTABLE_ERROR` to `IN_TRANSACTION` which is conceptually not acceptable.
This can be the root cause of some issues, with perhaps further future
changes by others.

3) If the poison pill record `r1` causes a transition to the error state
and then the next record `r2` requires adding a partition to the
transaction, the action fails due to being in the error state. In this
case, clearing errors during `commitTnx(CLEAR_SEND_ERROR)` is too late.
However, this case can NOT be the main concern as soon as KIP-890 is fully
implemented.


My suggestion is to solve the problem where it arises. If the transition to
the error state does not happen during `send()`, we do not need to clear
the error later. Therefore, instead of `CommitOption`, we can define a
`TxnSendOption` and prevent the `send()` method from going to the error
state in case 1) we're in a transaction and 2) the user asked for
IGONRE_SEND_ERRORS. For more clarity, you can take a look at the POC PR
.

Cheers,
Alieh


Re: [DISCUSS] KIP-1059: Enable the Producer flush() method to clear the latest send() error

2024-06-26 Thread Alieh Saeedi
Hi all,
thanks for the brilliant ideas.

The KIP is updated as follows:

-Regarding the word "latest”: I chose this word because in documentation of
the `commitTnx()`, it is clearly mentioned that the method throws the lates
exception of `send()`. BTW, I agree with you to change it to “any” since
“latest” can be confusing. The user may think, what about the pre-last
exception?!


-KIP name: updated to “Enable Producer to resolve send() method errors”.
Agree?


-Map is replaced by enum `CommitOption` as Andrew suggested. Thanks for all
the suggestions. I knew Map is not the best choice but hesitated to define
something new.


-The method javadocs: updated

Looking forward to your votes.

Bests,
Alieh

On Wed, Jun 26, 2024 at 10:52 AM Andrew Schofield 
wrote:

> Hi,
> Looking at your suggestions for the CommitOptions, I would be happy with
> either.
> I definitely prefer to the Map in the KIP.
>
> We also need to think about the other option where CLEAR_SEND_ERRORS
> hasn’t been
> specified, and leave ourselves space for other options in the future.
>
> If we use an enum, we need to give the existing behaviour a name. Maybe:
>
>public enum CommitOptions {
>   /**
>* Commits the ongoing transaction, flushing any unsent records
> before
>* actually committing the transaction. If any of the records sent
> in this transaction
>* hit unrecoverable errors, the transaction will not be committed.
>*/
>  NONE,
>
>   /**
>*  Commits the ongoing transaction, first clearing any errors from
> records already sent
>*  in this transaction and then flushing any unsent records before
> committing the transaction.
>*  If there are any unsent records flushed by this operation which
> hit unrecoverable errors,
>*  these errors will not be cleared and the transaction will not be
> committed.
>*  
>*  To ensure there are no unsent records, you must call {@link
> #flush()} before
>*  committing the transaction.
>*/
>   CLEAR_SEND_ERRORS;
>}
>
> The Javadoc for commitTransaction will have to be careful crafted.
>
> Thanks,
> Andrew
>
>
> > On 26 Jun 2024, at 03:23, Matthias J. Sax  wrote:
> >
> >>> I was also curious about this text:
>  The new method clears the latest error produced by
> `send(ProducerRecord)`
> >>> and transits the transaction back from the error state
> >
> > I agree. We should not say "latest" but "any".
> >
> >
> >
> >> Is it fair to say that we expect to encounter only one send response
> with
> >> an error that we clear on commitTransaction? Is that solving the
> problem?
> >
> > Yes, I think that is an accurate description.
> >
> >
> >> Speaking of documentation, is it confusing that the name of the KIP is
> no
> >> longer consistent with the approach the KIP takes?
> >
> > Sounds like an easy fix (we should not update the subject of the discuss
> email thread thought...)
> >
> >
> >
> >> looks like we said on the KIP that the options will be of the type
> Map >> ?> commitOptions. Would we want to define something more specific?
> >
> > I would also suggest to either pass an enum (as preferred by Andrew) or
> a newly define class `CommitOptions`.
> >
> > Something like:
> >
> >public enum CommitOptions {
> >/**
> > * Cleans any previous send errors, before committing the
> transaction.
> > * Note, if there are pending sends, error won't be cleared.
> > * To avoid pending sends, you need to call {@link #flush()}
> before committing the transaction.
> > */
> >CLEAR_SEND_ERRORS
> >}
> >
> >
> > Or similar for a class:
> >
> >public class CommitOptions {
> >/**
> > * Cleans any previous send errors, before committing the
> transaction.
> > * Note, if there are pending sends, error won't be cleared.
> > * To avoid pending sends, you need to call {@link #flush()}
> before committing the transaction.
> > */
> >public static CommitOptions clearSendErrors();
> >}
> >
> > Plus:
> >
> >public class KafkaProducer {
> >
> >/**
> > * 
> > *
> > * This method should only be called if there are no pending
> writes, ie, only after calling {@link #flush()).
> > * If there are any errors sending messaged to topics, these
> errors can be cleared by passing {@link CommitOptions#clearSendErrors},
> allowing the transaction to commit even in case of data loss during.
> > *
> > * If this method is used while there are pending sends,
> > * send error cannot be cleared.
> > */
> >public void commitTransaction(CommitOptions options);
> >}
> >
> > Or something like this -- we can discuss details about JavaDocs on the
> PR IMHO.
> >
> >
> >> What I
> >> really don't like (in any of the options), is that we cannot really
> >> document it in a way that articulates a value in the product.  There are
> >> tons of nuances 

Re: [DISCUSS] KIP-1059: Enable the Producer flush() method to clear the latest send() error

2024-06-26 Thread Andrew Schofield
Hi,
Looking at your suggestions for the CommitOptions, I would be happy with either.
I definitely prefer to the Map in the KIP.

We also need to think about the other option where CLEAR_SEND_ERRORS hasn’t been
specified, and leave ourselves space for other options in the future.

If we use an enum, we need to give the existing behaviour a name. Maybe:

   public enum CommitOptions {
  /**
   * Commits the ongoing transaction, flushing any unsent records before
   * actually committing the transaction. If any of the records sent in 
this transaction
   * hit unrecoverable errors, the transaction will not be committed.
   */
 NONE,

  /**
   *  Commits the ongoing transaction, first clearing any errors from 
records already sent
   *  in this transaction and then flushing any unsent records before 
committing the transaction.
   *  If there are any unsent records flushed by this operation which hit 
unrecoverable errors,
   *  these errors will not be cleared and the transaction will not be 
committed.
   *  
   *  To ensure there are no unsent records, you must call {@link #flush()} 
before
   *  committing the transaction.
   */
  CLEAR_SEND_ERRORS;
   }

The Javadoc for commitTransaction will have to be careful crafted.

Thanks,
Andrew


> On 26 Jun 2024, at 03:23, Matthias J. Sax  wrote:
>
>>> I was also curious about this text:
 The new method clears the latest error produced by `send(ProducerRecord)`
>>> and transits the transaction back from the error state
>
> I agree. We should not say "latest" but "any".
>
>
>
>> Is it fair to say that we expect to encounter only one send response with
>> an error that we clear on commitTransaction? Is that solving the problem?
>
> Yes, I think that is an accurate description.
>
>
>> Speaking of documentation, is it confusing that the name of the KIP is no
>> longer consistent with the approach the KIP takes?
>
> Sounds like an easy fix (we should not update the subject of the discuss 
> email thread thought...)
>
>
>
>> looks like we said on the KIP that the options will be of the type  
>> Map> ?> commitOptions. Would we want to define something more specific?
>
> I would also suggest to either pass an enum (as preferred by Andrew) or a 
> newly define class `CommitOptions`.
>
> Something like:
>
>public enum CommitOptions {
>/**
> * Cleans any previous send errors, before committing the transaction.
> * Note, if there are pending sends, error won't be cleared.
> * To avoid pending sends, you need to call {@link #flush()} before 
> committing the transaction.
> */
>CLEAR_SEND_ERRORS
>}
>
>
> Or similar for a class:
>
>public class CommitOptions {
>/**
> * Cleans any previous send errors, before committing the transaction.
> * Note, if there are pending sends, error won't be cleared.
> * To avoid pending sends, you need to call {@link #flush()} before 
> committing the transaction.
> */
>public static CommitOptions clearSendErrors();
>}
>
> Plus:
>
>public class KafkaProducer {
>
>/**
> * 
> *
> * This method should only be called if there are no pending writes, 
> ie, only after calling {@link #flush()).
> * If there are any errors sending messaged to topics, these errors 
> can be cleared by passing {@link CommitOptions#clearSendErrors}, allowing the 
> transaction to commit even in case of data loss during.
> *
> * If this method is used while there are pending sends,
> * send error cannot be cleared.
> */
>public void commitTransaction(CommitOptions options);
>}
>
> Or something like this -- we can discuss details about JavaDocs on the PR 
> IMHO.
>
>
>> What I
>> really don't like (in any of the options), is that we cannot really
>> document it in a way that articulates a value in the product.  There are
>> tons of nuances that require understanding some buggy behavior, then fixed
>> behavior, then an option to sometimes turn on buggy behavior, and etc.
>
> I would believe that users don't need to understand the full history of 
> events, and the propose JavaDocs should go a long way.
>
>
>
> -Matthias
>
>
> On 6/25/24 5:00 PM, Justine Olshan wrote:
>> Hey there,
>> I had a few questions about the update.
>> Looks like we said on the KIP that the options will be of the type  
>> Map> ?> commitOptions. Would we want to define something more specific?
>> I was also curious about this text:
>>> The new method clears the latest error produced by `send(ProducerRecord)`
>> and transits the transaction back from the error state
>> Is it fair to say that we expect to encounter only one send response with
>> an error that we clear on commitTransaction? Is that solving the problem?
>> I think we also need to be really careful about the documentation of this
>> method. It should be clear that setting this o

Re: [DISCUSS] KIP-1059: Enable the Producer flush() method to clear the latest send() error

2024-06-25 Thread Matthias J. Sax

I was also curious about this text:

The new method clears the latest error produced by `send(ProducerRecord)`

and transits the transaction back from the error state


I agree. We should not say "latest" but "any".




Is it fair to say that we expect to encounter only one send response with
an error that we clear on commitTransaction? Is that solving the problem?


Yes, I think that is an accurate description.



Speaking of documentation, is it confusing that the name of the KIP is no
longer consistent with the approach the KIP takes?


Sounds like an easy fix (we should not update the subject of the discuss 
email thread thought...)





looks like we said on the KIP that the options will be of the type  Map commitOptions. Would we want to define something more specific?


I would also suggest to either pass an enum (as preferred by Andrew) or 
a newly define class `CommitOptions`.


Something like:

public enum CommitOptions {
/**
 * Cleans any previous send errors, before committing the 
transaction.

 * Note, if there are pending sends, error won't be cleared.
 * To avoid pending sends, you need to call {@link #flush()} 
before committing the transaction.

 */
CLEAR_SEND_ERRORS
}


Or similar for a class:

public class CommitOptions {
/**
 * Cleans any previous send errors, before committing the 
transaction.

 * Note, if there are pending sends, error won't be cleared.
 * To avoid pending sends, you need to call {@link #flush()} 
before committing the transaction.

 */
public static CommitOptions clearSendErrors();
}

Plus:

public class KafkaProducer {

/**
 * 
 *
 * This method should only be called if there are no pending 
writes, ie, only after calling {@link #flush()).
 * If there are any errors sending messaged to topics, these 
errors can be cleared by passing {@link CommitOptions#clearSendErrors}, 
allowing the transaction to commit even in case of data loss during.

 *
 * If this method is used while there are pending sends,
 * send error cannot be cleared.
 */
public void commitTransaction(CommitOptions options);
}

Or something like this -- we can discuss details about JavaDocs on the 
PR IMHO.




What I
really don't like (in any of the options), is that we cannot really
document it in a way that articulates a value in the product.  There are
tons of nuances that require understanding some buggy behavior, then fixed
behavior, then an option to sometimes turn on buggy behavior, and etc.


I would believe that users don't need to understand the full history of 
events, and the propose JavaDocs should go a long way.




-Matthias


On 6/25/24 5:00 PM, Justine Olshan wrote:

Hey there,

I had a few questions about the update.
Looks like we said on the KIP that the options will be of the type  Map commitOptions. Would we want to define something more specific?

I was also curious about this text:

The new method clears the latest error produced by `send(ProducerRecord)`

and transits the transaction back from the error state

Is it fair to say that we expect to encounter only one send response with
an error that we clear on commitTransaction? Is that solving the problem?

I think we also need to be really careful about the documentation of this
method. It should be clear that setting this option will not do anything if
there is any inflight record when the method is called.
Speaking of documentation, is it confusing that the name of the KIP is no
longer consistent with the approach the KIP takes?

Thanks,
Justine

On Tue, Jun 25, 2024 at 5:08 AM Alieh Saeedi 
wrote:


Hi all,

Appreciation for maintaining the momentum of our discussion.


I see kinda consensus over the main points. It seems that we agreed on the
following:

1) Define the `commitTnx(commitOptions)` to clear the error.

2) Make the user explicitly call  `flush()` before
`commitTnx(commitOptions)`, if he determines ignoring errors.


I updated the KIP with the above-mentioned points. Please take a look. I am
sure it is not perfect yet, and there are/will be some open questions, but
if you agree, I will open voting as well. Of course, the discussion can
still carry on in this thread.


Cheers,

Alieh

On Tue, Jun 25, 2024 at 11:36 AM Chris Egerton 
wrote:


Hi Artem,

Yes, I completely agree that by default, special action shouldn't be
required from users to prevent transactions from being committed when one
or more records can't be sent. The behavior I was suggesting was only
relevant to the new API we're discussing where we allow users to
intentionally bypass that logic when invoking commitTransaction.

Cheers,

Chris

On Tue, Jun 25, 2024, 01:44 Artem Livshits 
wrote:


Hey folks,

Great discussion!

Re: throwing exceptions from send().  send() is documented to throw
KafkaException, so if the application doesn't handle it, it sh

Re: [DISCUSS] KIP-1059: Enable the Producer flush() method to clear the latest send() error

2024-06-25 Thread Justine Olshan
Hey there,

I had a few questions about the update.
Looks like we said on the KIP that the options will be of the type  Map commitOptions. Would we want to define something more specific?

I was also curious about this text:
> The new method clears the latest error produced by `send(ProducerRecord)`
and transits the transaction back from the error state

Is it fair to say that we expect to encounter only one send response with
an error that we clear on commitTransaction? Is that solving the problem?

I think we also need to be really careful about the documentation of this
method. It should be clear that setting this option will not do anything if
there is any inflight record when the method is called.
Speaking of documentation, is it confusing that the name of the KIP is no
longer consistent with the approach the KIP takes?

Thanks,
Justine

On Tue, Jun 25, 2024 at 5:08 AM Alieh Saeedi 
wrote:

> Hi all,
>
> Appreciation for maintaining the momentum of our discussion.
>
>
> I see kinda consensus over the main points. It seems that we agreed on the
> following:
>
> 1) Define the `commitTnx(commitOptions)` to clear the error.
>
> 2) Make the user explicitly call  `flush()` before
> `commitTnx(commitOptions)`, if he determines ignoring errors.
>
>
> I updated the KIP with the above-mentioned points. Please take a look. I am
> sure it is not perfect yet, and there are/will be some open questions, but
> if you agree, I will open voting as well. Of course, the discussion can
> still carry on in this thread.
>
>
> Cheers,
>
> Alieh
>
> On Tue, Jun 25, 2024 at 11:36 AM Chris Egerton 
> wrote:
>
> > Hi Artem,
> >
> > Yes, I completely agree that by default, special action shouldn't be
> > required from users to prevent transactions from being committed when one
> > or more records can't be sent. The behavior I was suggesting was only
> > relevant to the new API we're discussing where we allow users to
> > intentionally bypass that logic when invoking commitTransaction.
> >
> > Cheers,
> >
> > Chris
> >
> > On Tue, Jun 25, 2024, 01:44 Artem Livshits  > .invalid>
> > wrote:
> >
> > > Hey folks,
> > >
> > > Great discussion!
> > >
> > > Re: throwing exceptions from send().  send() is documented to throw
> > > KafkaException, so if the application doesn't handle it, it should be a
> > > bug.  Now, it does have a note that API exceptions wouldn't be thrown,
> > not
> > > sure if we have code that relies on that.  There is a reason exceptions
> > > have classes, they are designed to express a "class of errors" that can
> > be
> > > handled, so that we don't have to add a flag or a new method every time
> > we
> > > have a new exception to throw.  But if there is consensus that it's
> still
> > > too risky (especially if we have examples of code that gets broken),
> > then I
> > > agree that we shouldn't do it.
> > >
> > > Re: various ways to communicate semantics change.  If we must have 2
> > > different behaviors, I think passing options to "ignore errors" to
> > > commitTransaction is probably the most intuitive way to do it.  What I
> > > really don't like (in any of the options), is that we cannot really
> > > document it in a way that articulates a value in the product.  There
> are
> > > tons of nuances that require understanding some buggy behavior, then
> > fixed
> > > behavior, then an option to sometimes turn on buggy behavior, and etc.
> > >
> > > >  if a user invokes Producer::abortTransaction from within a producer
> > > callback today
> > >
> > > I think we would get invalid state exception.  Which we could probably
> > fix,
> > > but even if we supported it, I think it would be good if doing send +
> > > commit would lead to aborted transaction without special action from
> the
> > > application -- the simple things should be really simple, any failure
> > > during send or commit should abort send + commit sequence without
> special
> > > handling.
> > >
> > > -Artem
> > >
> > > On Mon, Jun 24, 2024 at 6:37 PM Chris Egerton  >
> > > wrote:
> > >
> > > > One quick thought: if a user invokes Producer::abortTransaction from
> > > within
> > > > a producer callback today, even in the midst of an ongoing call to
> > > > Producer::commitTransaction, what is the behavior? Would it be
> > reasonable
> > > > to support this behavior as a way to allow error handling to take
> place
> > > > during implicit flushes, via producer callback?
> > > >
> > > > On Mon, Jun 24, 2024 at 9:21 PM Matthias J. Sax 
> > > wrote:
> > > >
> > > > > My point it, that it does not seem to be safe to allow users to
> > ignore
> > > > > errors with an implicit flush, and I think it's better to only
> allow
> > it
> > > > > with (ie, after) an explicit flush().
> > > > >
> > > > > My reasoning is, that users should make a decision to ignore errors
> > or
> > > > > not, before calling `commitTx()`, but after inspecting all
> potential
> > > > > send errors. With an implicit flush, users need to "blindly" decide
> > to
> > > > > ignore sen

Re: [DISCUSS] KIP-1059: Enable the Producer flush() method to clear the latest send() error

2024-06-25 Thread Alieh Saeedi
Hi all,

Appreciation for maintaining the momentum of our discussion.


I see kinda consensus over the main points. It seems that we agreed on the
following:

1) Define the `commitTnx(commitOptions)` to clear the error.

2) Make the user explicitly call  `flush()` before
`commitTnx(commitOptions)`, if he determines ignoring errors.


I updated the KIP with the above-mentioned points. Please take a look. I am
sure it is not perfect yet, and there are/will be some open questions, but
if you agree, I will open voting as well. Of course, the discussion can
still carry on in this thread.


Cheers,

Alieh

On Tue, Jun 25, 2024 at 11:36 AM Chris Egerton 
wrote:

> Hi Artem,
>
> Yes, I completely agree that by default, special action shouldn't be
> required from users to prevent transactions from being committed when one
> or more records can't be sent. The behavior I was suggesting was only
> relevant to the new API we're discussing where we allow users to
> intentionally bypass that logic when invoking commitTransaction.
>
> Cheers,
>
> Chris
>
> On Tue, Jun 25, 2024, 01:44 Artem Livshits  .invalid>
> wrote:
>
> > Hey folks,
> >
> > Great discussion!
> >
> > Re: throwing exceptions from send().  send() is documented to throw
> > KafkaException, so if the application doesn't handle it, it should be a
> > bug.  Now, it does have a note that API exceptions wouldn't be thrown,
> not
> > sure if we have code that relies on that.  There is a reason exceptions
> > have classes, they are designed to express a "class of errors" that can
> be
> > handled, so that we don't have to add a flag or a new method every time
> we
> > have a new exception to throw.  But if there is consensus that it's still
> > too risky (especially if we have examples of code that gets broken),
> then I
> > agree that we shouldn't do it.
> >
> > Re: various ways to communicate semantics change.  If we must have 2
> > different behaviors, I think passing options to "ignore errors" to
> > commitTransaction is probably the most intuitive way to do it.  What I
> > really don't like (in any of the options), is that we cannot really
> > document it in a way that articulates a value in the product.  There are
> > tons of nuances that require understanding some buggy behavior, then
> fixed
> > behavior, then an option to sometimes turn on buggy behavior, and etc.
> >
> > >  if a user invokes Producer::abortTransaction from within a producer
> > callback today
> >
> > I think we would get invalid state exception.  Which we could probably
> fix,
> > but even if we supported it, I think it would be good if doing send +
> > commit would lead to aborted transaction without special action from the
> > application -- the simple things should be really simple, any failure
> > during send or commit should abort send + commit sequence without special
> > handling.
> >
> > -Artem
> >
> > On Mon, Jun 24, 2024 at 6:37 PM Chris Egerton 
> > wrote:
> >
> > > One quick thought: if a user invokes Producer::abortTransaction from
> > within
> > > a producer callback today, even in the midst of an ongoing call to
> > > Producer::commitTransaction, what is the behavior? Would it be
> reasonable
> > > to support this behavior as a way to allow error handling to take place
> > > during implicit flushes, via producer callback?
> > >
> > > On Mon, Jun 24, 2024 at 9:21 PM Matthias J. Sax 
> > wrote:
> > >
> > > > My point it, that it does not seem to be safe to allow users to
> ignore
> > > > errors with an implicit flush, and I think it's better to only allow
> it
> > > > with (ie, after) an explicit flush().
> > > >
> > > > My reasoning is, that users should make a decision to ignore errors
> or
> > > > not, before calling `commitTx()`, but after inspecting all potential
> > > > send errors. With an implicit flush, users need to "blindly" decide
> to
> > > > ignore send errors, because there are pending sends and potential
> > errors
> > > > are not known yet, when calling `commitTx()`.
> > > >
> > > >
> > > >
> > > > > In the documentation of commitTransaction, we say if any send
> throws
> > an
> > > > > error, commitTransaction will too.
> > > >
> > > > Yes. And I think we should keep it this way for an implicit flush.
> With
> > > > an explicit flush, `commitTransaction()` cannot encounter any send
> > > > errors any longer.
> > > >
> > > >
> > > >
> > > > > It says that all callbacks will be executed, but we ignore the
> errors
> > > of
> > > > > the callbacks.
> > > >
> > > > Ah. Thanks for pointing this out. For this case it's even worse (for
> > > > case (2)), because the user cannot inspect any errors and make any
> > > > decision to ignore or not during an implicit flush...
> > > >
> > > >
> > > >
> > > > > We shouldn't be relying on errors in the callback unless we are
> > > > > calling flush, which we can still do. It seems this has always been
> > the
> > > > > case as well.
> > > >
> > > > Yes, has always been this way, and my point is to keep it this way
> > > > 

Re: [DISCUSS] KIP-1059: Enable the Producer flush() method to clear the latest send() error

2024-06-25 Thread Chris Egerton
Hi Artem,

Yes, I completely agree that by default, special action shouldn't be
required from users to prevent transactions from being committed when one
or more records can't be sent. The behavior I was suggesting was only
relevant to the new API we're discussing where we allow users to
intentionally bypass that logic when invoking commitTransaction.

Cheers,

Chris

On Tue, Jun 25, 2024, 01:44 Artem Livshits 
wrote:

> Hey folks,
>
> Great discussion!
>
> Re: throwing exceptions from send().  send() is documented to throw
> KafkaException, so if the application doesn't handle it, it should be a
> bug.  Now, it does have a note that API exceptions wouldn't be thrown, not
> sure if we have code that relies on that.  There is a reason exceptions
> have classes, they are designed to express a "class of errors" that can be
> handled, so that we don't have to add a flag or a new method every time we
> have a new exception to throw.  But if there is consensus that it's still
> too risky (especially if we have examples of code that gets broken), then I
> agree that we shouldn't do it.
>
> Re: various ways to communicate semantics change.  If we must have 2
> different behaviors, I think passing options to "ignore errors" to
> commitTransaction is probably the most intuitive way to do it.  What I
> really don't like (in any of the options), is that we cannot really
> document it in a way that articulates a value in the product.  There are
> tons of nuances that require understanding some buggy behavior, then fixed
> behavior, then an option to sometimes turn on buggy behavior, and etc.
>
> >  if a user invokes Producer::abortTransaction from within a producer
> callback today
>
> I think we would get invalid state exception.  Which we could probably fix,
> but even if we supported it, I think it would be good if doing send +
> commit would lead to aborted transaction without special action from the
> application -- the simple things should be really simple, any failure
> during send or commit should abort send + commit sequence without special
> handling.
>
> -Artem
>
> On Mon, Jun 24, 2024 at 6:37 PM Chris Egerton 
> wrote:
>
> > One quick thought: if a user invokes Producer::abortTransaction from
> within
> > a producer callback today, even in the midst of an ongoing call to
> > Producer::commitTransaction, what is the behavior? Would it be reasonable
> > to support this behavior as a way to allow error handling to take place
> > during implicit flushes, via producer callback?
> >
> > On Mon, Jun 24, 2024 at 9:21 PM Matthias J. Sax 
> wrote:
> >
> > > My point it, that it does not seem to be safe to allow users to ignore
> > > errors with an implicit flush, and I think it's better to only allow it
> > > with (ie, after) an explicit flush().
> > >
> > > My reasoning is, that users should make a decision to ignore errors or
> > > not, before calling `commitTx()`, but after inspecting all potential
> > > send errors. With an implicit flush, users need to "blindly" decide to
> > > ignore send errors, because there are pending sends and potential
> errors
> > > are not known yet, when calling `commitTx()`.
> > >
> > >
> > >
> > > > In the documentation of commitTransaction, we say if any send throws
> an
> > > > error, commitTransaction will too.
> > >
> > > Yes. And I think we should keep it this way for an implicit flush. With
> > > an explicit flush, `commitTransaction()` cannot encounter any send
> > > errors any longer.
> > >
> > >
> > >
> > > > It says that all callbacks will be executed, but we ignore the errors
> > of
> > > > the callbacks.
> > >
> > > Ah. Thanks for pointing this out. For this case it's even worse (for
> > > case (2)), because the user cannot inspect any errors and make any
> > > decision to ignore or not during an implicit flush...
> > >
> > >
> > >
> > > > We shouldn't be relying on errors in the callback unless we are
> > > > calling flush, which we can still do. It seems this has always been
> the
> > > > case as well.
> > >
> > > Yes, has always been this way, and my point is to keep it this way
> > > (option (2) would change it), and not start to allow to ignore errors
> > > with an implicit flush.
> > >
> > >
> > >
> > > -Matthias
> > >
> > >
> > >
> > > On 6/24/24 4:57 PM, Justine Olshan wrote:
> > > > Transaction verification is a concept from KIP-890 referring to the
> > > > verification that a partition has been added to the transaction. It's
> > > not a
> > > > huge deal, but maybe we don't want to overload the terminology.
> > > >
> > > > For option 2, I was a little confused by this
> > > >
> > > >>when commitTx is called, there is still pending Futures and not
> > > > all Callbacks are executed yet -- with the implicit flush, we know
> that
> > > > all Callbacks are executed, but even for this case, the user could
> only
> > > > throw an exception inside the Callback to stop the TX to eventually
> > > > commit -- Futures cannot be used to make a decision to ignore error
>

Re: [DISCUSS] KIP-1059: Enable the Producer flush() method to clear the latest send() error

2024-06-24 Thread Artem Livshits
Hey folks,

Great discussion!

Re: throwing exceptions from send().  send() is documented to throw
KafkaException, so if the application doesn't handle it, it should be a
bug.  Now, it does have a note that API exceptions wouldn't be thrown, not
sure if we have code that relies on that.  There is a reason exceptions
have classes, they are designed to express a "class of errors" that can be
handled, so that we don't have to add a flag or a new method every time we
have a new exception to throw.  But if there is consensus that it's still
too risky (especially if we have examples of code that gets broken), then I
agree that we shouldn't do it.

Re: various ways to communicate semantics change.  If we must have 2
different behaviors, I think passing options to "ignore errors" to
commitTransaction is probably the most intuitive way to do it.  What I
really don't like (in any of the options), is that we cannot really
document it in a way that articulates a value in the product.  There are
tons of nuances that require understanding some buggy behavior, then fixed
behavior, then an option to sometimes turn on buggy behavior, and etc.

>  if a user invokes Producer::abortTransaction from within a producer
callback today

I think we would get invalid state exception.  Which we could probably fix,
but even if we supported it, I think it would be good if doing send +
commit would lead to aborted transaction without special action from the
application -- the simple things should be really simple, any failure
during send or commit should abort send + commit sequence without special
handling.

-Artem

On Mon, Jun 24, 2024 at 6:37 PM Chris Egerton 
wrote:

> One quick thought: if a user invokes Producer::abortTransaction from within
> a producer callback today, even in the midst of an ongoing call to
> Producer::commitTransaction, what is the behavior? Would it be reasonable
> to support this behavior as a way to allow error handling to take place
> during implicit flushes, via producer callback?
>
> On Mon, Jun 24, 2024 at 9:21 PM Matthias J. Sax  wrote:
>
> > My point it, that it does not seem to be safe to allow users to ignore
> > errors with an implicit flush, and I think it's better to only allow it
> > with (ie, after) an explicit flush().
> >
> > My reasoning is, that users should make a decision to ignore errors or
> > not, before calling `commitTx()`, but after inspecting all potential
> > send errors. With an implicit flush, users need to "blindly" decide to
> > ignore send errors, because there are pending sends and potential errors
> > are not known yet, when calling `commitTx()`.
> >
> >
> >
> > > In the documentation of commitTransaction, we say if any send throws an
> > > error, commitTransaction will too.
> >
> > Yes. And I think we should keep it this way for an implicit flush. With
> > an explicit flush, `commitTransaction()` cannot encounter any send
> > errors any longer.
> >
> >
> >
> > > It says that all callbacks will be executed, but we ignore the errors
> of
> > > the callbacks.
> >
> > Ah. Thanks for pointing this out. For this case it's even worse (for
> > case (2)), because the user cannot inspect any errors and make any
> > decision to ignore or not during an implicit flush...
> >
> >
> >
> > > We shouldn't be relying on errors in the callback unless we are
> > > calling flush, which we can still do. It seems this has always been the
> > > case as well.
> >
> > Yes, has always been this way, and my point is to keep it this way
> > (option (2) would change it), and not start to allow to ignore errors
> > with an implicit flush.
> >
> >
> >
> > -Matthias
> >
> >
> >
> > On 6/24/24 4:57 PM, Justine Olshan wrote:
> > > Transaction verification is a concept from KIP-890 referring to the
> > > verification that a partition has been added to the transaction. It's
> > not a
> > > huge deal, but maybe we don't want to overload the terminology.
> > >
> > > For option 2, I was a little confused by this
> > >
> > >>when commitTx is called, there is still pending Futures and not
> > > all Callbacks are executed yet -- with the implicit flush, we know that
> > > all Callbacks are executed, but even for this case, the user could only
> > > throw an exception inside the Callback to stop the TX to eventually
> > > commit -- Futures cannot be used to make a decision to ignore error and
> > > commit or not.
> > >
> > > In the documentation of commitTransaction, we say if any send throws an
> > > error, commitTransaction will too.
> > >
> > > *Further, if any of the {@link #send(ProducerRecord)} calls which were
> > part
> > > of the transaction hit irrecoverable errors, this method will throw the
> > > last received exception immediately and the transaction will not be
> > > committed.*
> > >
> > > It says that all callbacks will be executed, but we ignore the errors
> of
> > > the callbacks.
> > >
> > > *If the transaction is committed successfully and this method returns
> > > without throwing an exceptio

Re: [DISCUSS] KIP-1059: Enable the Producer flush() method to clear the latest send() error

2024-06-24 Thread Chris Egerton
One quick thought: if a user invokes Producer::abortTransaction from within
a producer callback today, even in the midst of an ongoing call to
Producer::commitTransaction, what is the behavior? Would it be reasonable
to support this behavior as a way to allow error handling to take place
during implicit flushes, via producer callback?

On Mon, Jun 24, 2024 at 9:21 PM Matthias J. Sax  wrote:

> My point it, that it does not seem to be safe to allow users to ignore
> errors with an implicit flush, and I think it's better to only allow it
> with (ie, after) an explicit flush().
>
> My reasoning is, that users should make a decision to ignore errors or
> not, before calling `commitTx()`, but after inspecting all potential
> send errors. With an implicit flush, users need to "blindly" decide to
> ignore send errors, because there are pending sends and potential errors
> are not known yet, when calling `commitTx()`.
>
>
>
> > In the documentation of commitTransaction, we say if any send throws an
> > error, commitTransaction will too.
>
> Yes. And I think we should keep it this way for an implicit flush. With
> an explicit flush, `commitTransaction()` cannot encounter any send
> errors any longer.
>
>
>
> > It says that all callbacks will be executed, but we ignore the errors of
> > the callbacks.
>
> Ah. Thanks for pointing this out. For this case it's even worse (for
> case (2)), because the user cannot inspect any errors and make any
> decision to ignore or not during an implicit flush...
>
>
>
> > We shouldn't be relying on errors in the callback unless we are
> > calling flush, which we can still do. It seems this has always been the
> > case as well.
>
> Yes, has always been this way, and my point is to keep it this way
> (option (2) would change it), and not start to allow to ignore errors
> with an implicit flush.
>
>
>
> -Matthias
>
>
>
> On 6/24/24 4:57 PM, Justine Olshan wrote:
> > Transaction verification is a concept from KIP-890 referring to the
> > verification that a partition has been added to the transaction. It's
> not a
> > huge deal, but maybe we don't want to overload the terminology.
> >
> > For option 2, I was a little confused by this
> >
> >>when commitTx is called, there is still pending Futures and not
> > all Callbacks are executed yet -- with the implicit flush, we know that
> > all Callbacks are executed, but even for this case, the user could only
> > throw an exception inside the Callback to stop the TX to eventually
> > commit -- Futures cannot be used to make a decision to ignore error and
> > commit or not.
> >
> > In the documentation of commitTransaction, we say if any send throws an
> > error, commitTransaction will too.
> >
> > *Further, if any of the {@link #send(ProducerRecord)} calls which were
> part
> > of the transaction hit irrecoverable errors, this method will throw the
> > last received exception immediately and the transaction will not be
> > committed.*
> >
> > It says that all callbacks will be executed, but we ignore the errors of
> > the callbacks.
> >
> > *If the transaction is committed successfully and this method returns
> > without throwing an exception, it is guaranteed that all {@link Callback
> > callbacks} for records in the transaction will have been invoked and
> > completed. Note that exceptions thrown by callbacks are ignored; the
> > producer proceeds to commit the transaction in any case.*
> >
> > Is it fair to say though that for the send errors, we can choose to
> ignore
> > them? II wasn't understanding where the callbacks come in with your
> > comment. We shouldn't be relying on errors in the callback unless we are
> > calling flush, which we can still do. It seems this has always been the
> > case as well.
> >
> > Justine
> >
> > On Mon, Jun 24, 2024 at 11:07 AM Andrew Schofield <
> andrew_schofi...@live.com>
> > wrote:
> >
> >> Agreed. Options 1 and 3 are safe. Option 2 is not. I’d be happy with 3a
> as
> >> the way.
> >>
> >> I suggest “TRANSACTION VERIFIED”.
> >>
> >> There isn’t really precedent for options in the producer API. We could
> use
> >> an enum,
> >> which is easy to use and not very future-proof. Or we could use a class
> >> like the
> >> admin API does, which is cumbersome and flexible.
> >>
> >>CommitTransactionOptions.TRANSACTION_VERIFIED
> >>
> >> or
> >>
> >>public class CommitTransactionOptions {
> >>  public CommitTransactionOptions();
> >>
> >>  CommitTransactionOptions transactionVerified(boolean
> >> transactionVerified);
> >>
> >>  boolean transactionVerified();
> >>}
> >>
> >>
> >> Then 3b is:
> >>
> >> send(…)
> >> send(…)
> >> flush()
> >> commitTransaction(new
> >> CommitTransactionOptions().transactionVerified(true))
> >>
> >>
> >> I’d tend towards the enum here because I doubt we need as much
> flexibility
> >> as the admin API requires.
> >>
> >> Thanks,
> >> Andrew
> >>
> >>
> >>> On 24 Jun 2024, at 18:39, Matthias J. Sax  wrote:
> >>>
> >>> I am ok either way 

Re: [DISCUSS] KIP-1059: Enable the Producer flush() method to clear the latest send() error

2024-06-24 Thread Matthias J. Sax
My point it, that it does not seem to be safe to allow users to ignore 
errors with an implicit flush, and I think it's better to only allow it 
with (ie, after) an explicit flush().


My reasoning is, that users should make a decision to ignore errors or 
not, before calling `commitTx()`, but after inspecting all potential 
send errors. With an implicit flush, users need to "blindly" decide to 
ignore send errors, because there are pending sends and potential errors 
are not known yet, when calling `commitTx()`.





In the documentation of commitTransaction, we say if any send throws an
error, commitTransaction will too.


Yes. And I think we should keep it this way for an implicit flush. With 
an explicit flush, `commitTransaction()` cannot encounter any send 
errors any longer.





It says that all callbacks will be executed, but we ignore the errors of
the callbacks.


Ah. Thanks for pointing this out. For this case it's even worse (for 
case (2)), because the user cannot inspect any errors and make any 
decision to ignore or not during an implicit flush...





We shouldn't be relying on errors in the callback unless we are
calling flush, which we can still do. It seems this has always been the
case as well.


Yes, has always been this way, and my point is to keep it this way 
(option (2) would change it), and not start to allow to ignore errors 
with an implicit flush.




-Matthias



On 6/24/24 4:57 PM, Justine Olshan wrote:

Transaction verification is a concept from KIP-890 referring to the
verification that a partition has been added to the transaction. It's not a
huge deal, but maybe we don't want to overload the terminology.

For option 2, I was a little confused by this


   when commitTx is called, there is still pending Futures and not

all Callbacks are executed yet -- with the implicit flush, we know that
all Callbacks are executed, but even for this case, the user could only
throw an exception inside the Callback to stop the TX to eventually
commit -- Futures cannot be used to make a decision to ignore error and
commit or not.

In the documentation of commitTransaction, we say if any send throws an
error, commitTransaction will too.

*Further, if any of the {@link #send(ProducerRecord)} calls which were part
of the transaction hit irrecoverable errors, this method will throw the
last received exception immediately and the transaction will not be
committed.*

It says that all callbacks will be executed, but we ignore the errors of
the callbacks.

*If the transaction is committed successfully and this method returns
without throwing an exception, it is guaranteed that all {@link Callback
callbacks} for records in the transaction will have been invoked and
completed. Note that exceptions thrown by callbacks are ignored; the
producer proceeds to commit the transaction in any case.*

Is it fair to say though that for the send errors, we can choose to ignore
them? II wasn't understanding where the callbacks come in with your
comment. We shouldn't be relying on errors in the callback unless we are
calling flush, which we can still do. It seems this has always been the
case as well.

Justine

On Mon, Jun 24, 2024 at 11:07 AM Andrew Schofield 
wrote:


Agreed. Options 1 and 3 are safe. Option 2 is not. I’d be happy with 3a as
the way.

I suggest “TRANSACTION VERIFIED”.

There isn’t really precedent for options in the producer API. We could use
an enum,
which is easy to use and not very future-proof. Or we could use a class
like the
admin API does, which is cumbersome and flexible.

   CommitTransactionOptions.TRANSACTION_VERIFIED

or

   public class CommitTransactionOptions {
 public CommitTransactionOptions();

 CommitTransactionOptions transactionVerified(boolean
transactionVerified);

 boolean transactionVerified();
   }


Then 3b is:

send(…)
send(…)
flush()
commitTransaction(new
CommitTransactionOptions().transactionVerified(true))


I’d tend towards the enum here because I doubt we need as much flexibility
as the admin API requires.

Thanks,
Andrew



On 24 Jun 2024, at 18:39, Matthias J. Sax  wrote:

I am ok either way (ie, flush or commit), but I think we need to define

exact semantics, and I think there is some subtle thing to consider:




1) flush(Options)

Example:

  send(...)
  send(...)

  flush(ignoreErrors)

  // at this point, we know that all Futures are completed and all

Callbacks are executed, and we can assume that all user code checking for
errors did execute, before `commitTx` is called


  // I consider this option as safe

  commitTx()


2) commitTx(Option)

Example:

  send(...)
  send(...)

  // when commitTx is called, there is still pending Futures and not all

Callbacks are executed yet -- with the implicit flush, we know that all
Callbacks are executed, but even for this case, the user could only throw
an exception inside the Callback to stop the TX to eventually commit --
Futures cannot be used to make a decision to ignore error an

Re: [DISCUSS] KIP-1059: Enable the Producer flush() method to clear the latest send() error

2024-06-24 Thread Justine Olshan
Transaction verification is a concept from KIP-890 referring to the
verification that a partition has been added to the transaction. It's not a
huge deal, but maybe we don't want to overload the terminology.

For option 2, I was a little confused by this

>   when commitTx is called, there is still pending Futures and not
all Callbacks are executed yet -- with the implicit flush, we know that
all Callbacks are executed, but even for this case, the user could only
throw an exception inside the Callback to stop the TX to eventually
commit -- Futures cannot be used to make a decision to ignore error and
commit or not.

In the documentation of commitTransaction, we say if any send throws an
error, commitTransaction will too.

*Further, if any of the {@link #send(ProducerRecord)} calls which were part
of the transaction hit irrecoverable errors, this method will throw the
last received exception immediately and the transaction will not be
committed.*

It says that all callbacks will be executed, but we ignore the errors of
the callbacks.

*If the transaction is committed successfully and this method returns
without throwing an exception, it is guaranteed that all {@link Callback
callbacks} for records in the transaction will have been invoked and
completed. Note that exceptions thrown by callbacks are ignored; the
producer proceeds to commit the transaction in any case.*

Is it fair to say though that for the send errors, we can choose to ignore
them? II wasn't understanding where the callbacks come in with your
comment. We shouldn't be relying on errors in the callback unless we are
calling flush, which we can still do. It seems this has always been the
case as well.

Justine

On Mon, Jun 24, 2024 at 11:07 AM Andrew Schofield 
wrote:

> Agreed. Options 1 and 3 are safe. Option 2 is not. I’d be happy with 3a as
> the way.
>
> I suggest “TRANSACTION VERIFIED”.
>
> There isn’t really precedent for options in the producer API. We could use
> an enum,
> which is easy to use and not very future-proof. Or we could use a class
> like the
> admin API does, which is cumbersome and flexible.
>
>   CommitTransactionOptions.TRANSACTION_VERIFIED
>
> or
>
>   public class CommitTransactionOptions {
> public CommitTransactionOptions();
>
> CommitTransactionOptions transactionVerified(boolean
> transactionVerified);
>
> boolean transactionVerified();
>   }
>
>
> Then 3b is:
>
>send(…)
>send(…)
>flush()
>commitTransaction(new
> CommitTransactionOptions().transactionVerified(true))
>
>
> I’d tend towards the enum here because I doubt we need as much flexibility
> as the admin API requires.
>
> Thanks,
> Andrew
>
>
> > On 24 Jun 2024, at 18:39, Matthias J. Sax  wrote:
> >
> > I am ok either way (ie, flush or commit), but I think we need to define
> exact semantics, and I think there is some subtle thing to consider:
> >
> >
> >
> > 1) flush(Options)
> >
> > Example:
> >
> >  send(...)
> >  send(...)
> >
> >  flush(ignoreErrors)
> >
> >  // at this point, we know that all Futures are completed and all
> Callbacks are executed, and we can assume that all user code checking for
> errors did execute, before `commitTx` is called
> >
> >  // I consider this option as safe
> >
> >  commitTx()
> >
> >
> > 2) commitTx(Option)
> >
> > Example:
> >
> >  send(...)
> >  send(...)
> >
> >  // when commitTx is called, there is still pending Futures and not all
> Callbacks are executed yet -- with the implicit flush, we know that all
> Callbacks are executed, but even for this case, the user could only throw
> an exception inside the Callback to stop the TX to eventually commit --
> Futures cannot be used to make a decision to ignore error and commit or not.
> >
> >  // I consider this option not as safe
> >
> >  commitTx(igrnoreErrors)
> >
> >
> >
> > 3a) required flush + commitTx(Option)
> >
> > Example:
> >
> >  send(...)
> >  send(...)
> >
> >  flush()
> >
> >  // at this point, we know that all Future are completed and all
> Callbacks are executed, and we can assume that all user code checking for
> error did execute, before `commitTx` is called
> >
> >  // I consider this option as safe
> >
> >  commitTx(ignoreErrors)
> >
> >
> > 3b) missing flush + commitTx(Option)
> >
> > Example:
> >
> >  send(...)
> >  send(...)
> >
> >  // as flush() was not called explicitly, we should ignore
> `ignoreErrors` flag and always throw an exception if the producer is in
> error state, because we cannot be sure that the user did all required check
> for error handling
> >
> >  commitTx(ignoreErrors)
> >
> >
> >
> > The only issue with option (3) is, that it's more complex and semantics
> are more subtle. But it might be the a good (necessary?) bridge between (1)
> and (2): (3) is semantically sound (we ignore errors via passing a flag
> into commitTx() instead of flush()), and at the same time safe (we force
> users to explicitly flush() and [hopefully] do proper error handling, and
> don't rely in am implicit flush() during com

Re: [DISCUSS] KIP-1059: Enable the Producer flush() method to clear the latest send() error

2024-06-24 Thread Andrew Schofield
Agreed. Options 1 and 3 are safe. Option 2 is not. I’d be happy with 3a as the 
way.

I suggest “TRANSACTION VERIFIED”.

There isn’t really precedent for options in the producer API. We could use an 
enum,
which is easy to use and not very future-proof. Or we could use a class like the
admin API does, which is cumbersome and flexible.

  CommitTransactionOptions.TRANSACTION_VERIFIED

or

  public class CommitTransactionOptions {
public CommitTransactionOptions();

CommitTransactionOptions transactionVerified(boolean transactionVerified);

boolean transactionVerified();
  }


Then 3b is:

   send(…)
   send(…)
   flush()
   commitTransaction(new CommitTransactionOptions().transactionVerified(true))


I’d tend towards the enum here because I doubt we need as much flexibility
as the admin API requires.

Thanks,
Andrew


> On 24 Jun 2024, at 18:39, Matthias J. Sax  wrote:
>
> I am ok either way (ie, flush or commit), but I think we need to define exact 
> semantics, and I think there is some subtle thing to consider:
>
>
>
> 1) flush(Options)
>
> Example:
>
>  send(...)
>  send(...)
>
>  flush(ignoreErrors)
>
>  // at this point, we know that all Futures are completed and all Callbacks 
> are executed, and we can assume that all user code checking for errors did 
> execute, before `commitTx` is called
>
>  // I consider this option as safe
>
>  commitTx()
>
>
> 2) commitTx(Option)
>
> Example:
>
>  send(...)
>  send(...)
>
>  // when commitTx is called, there is still pending Futures and not all 
> Callbacks are executed yet -- with the implicit flush, we know that all 
> Callbacks are executed, but even for this case, the user could only throw an 
> exception inside the Callback to stop the TX to eventually commit -- Futures 
> cannot be used to make a decision to ignore error and commit or not.
>
>  // I consider this option not as safe
>
>  commitTx(igrnoreErrors)
>
>
>
> 3a) required flush + commitTx(Option)
>
> Example:
>
>  send(...)
>  send(...)
>
>  flush()
>
>  // at this point, we know that all Future are completed and all Callbacks 
> are executed, and we can assume that all user code checking for error did 
> execute, before `commitTx` is called
>
>  // I consider this option as safe
>
>  commitTx(ignoreErrors)
>
>
> 3b) missing flush + commitTx(Option)
>
> Example:
>
>  send(...)
>  send(...)
>
>  // as flush() was not called explicitly, we should ignore `ignoreErrors` 
> flag and always throw an exception if the producer is in error state, because 
> we cannot be sure that the user did all required check for error handling
>
>  commitTx(ignoreErrors)
>
>
>
> The only issue with option (3) is, that it's more complex and semantics are 
> more subtle. But it might be the a good (necessary?) bridge between (1) and 
> (2): (3) is semantically sound (we ignore errors via passing a flag into 
> commitTx() instead of flush()), and at the same time safe (we force users to 
> explicitly flush() and [hopefully] do proper error handling, and don't rely 
> in am implicit flush() during commitTx() which might be error prone).
>
> (Also need to find a good and descriptive name for the flag we pass into 
> `commitTx()` for this case.)
>
>
> -Matthias
>
>
>
> On 6/24/24 8:51 AM, Andrew Schofield wrote:
>> Hi Chris,
>> That works for me too. I slightly prefer an option on flush(), but what you 
>> suggested
>> works too.
>> Thanks,
>> Andrew
>>> On 24 Jun 2024, at 15:14, Chris Egerton  wrote:
>>>
>>> Hi Andrew,
>>>
>>> I like a lot of what you said, but I still believe it's better to override
>>> commitTransaction than flush. Users will already have to manually opt in to
>>> ignoring errors encountered during transactions, and we can document
>>> recommended usage (i.e., explicitly invoking flush() before invoking
>>> commitTransaction(ignoreRecordErrors)) in the newly-introduced method. I
>>> don't believe it's worth the increased cognitive load on users with
>>> non-transactional producers to introduce an overloaded flush() variant.
>>>
>>> Cheers,
>>>
>>> Chris
>>>
>>> On Mon, Jun 24, 2024 at 9:39 AM Andrew Schofield 
>>> wrote:
>>>
 Hi Alieh,
 Thanks for driving this. Unfortunately, there are many parts of the API
 which
 are a bit unfortunate and it’s tricky to make small improvements that
 don’t have
 downsides.

 I don’t like the idea of using a configuration because configuration is
 often
 outside the application and changing the behaviour of someone else’s
 application
 without understanding it is risky. Anything which embeds a transactional
 producer
 could have its behaviour changed unexpectedly.

 It would be been much nicer if send() didn’t fail silently and change the
 transaction
 state. But, because it’s an asynchronous operation, I don’t really think
 we can
 just make it throw all exceptions, even though I really think that
 `send()` is the
 method with the problem here.

 The contract of 

Re: [DISCUSS] KIP-1059: Enable the Producer flush() method to clear the latest send() error

2024-06-24 Thread Matthias J. Sax
I am ok either way (ie, flush or commit), but I think we need to define 
exact semantics, and I think there is some subtle thing to consider:




1) flush(Options)

Example:

  send(...)
  send(...)

  flush(ignoreErrors)

  // at this point, we know that all Futures are completed and all 
Callbacks are executed, and we can assume that all user code checking 
for errors did execute, before `commitTx` is called


  // I consider this option as safe

  commitTx()


2) commitTx(Option)

Example:

  send(...)
  send(...)

  // when commitTx is called, there is still pending Futures and not 
all Callbacks are executed yet -- with the implicit flush, we know that 
all Callbacks are executed, but even for this case, the user could only 
throw an exception inside the Callback to stop the TX to eventually 
commit -- Futures cannot be used to make a decision to ignore error and 
commit or not.


  // I consider this option not as safe

  commitTx(igrnoreErrors)



3a) required flush + commitTx(Option)

Example:

  send(...)
  send(...)

  flush()

  // at this point, we know that all Future are completed and all 
Callbacks are executed, and we can assume that all user code checking 
for error did execute, before `commitTx` is called


  // I consider this option as safe

  commitTx(ignoreErrors)


3b) missing flush + commitTx(Option)

Example:

  send(...)
  send(...)

  // as flush() was not called explicitly, we should ignore 
`ignoreErrors` flag and always throw an exception if the producer is in 
error state, because we cannot be sure that the user did all required 
check for error handling


  commitTx(ignoreErrors)



The only issue with option (3) is, that it's more complex and semantics 
are more subtle. But it might be the a good (necessary?) bridge between 
(1) and (2): (3) is semantically sound (we ignore errors via passing a 
flag into commitTx() instead of flush()), and at the same time safe (we 
force users to explicitly flush() and [hopefully] do proper error 
handling, and don't rely in am implicit flush() during commitTx() which 
might be error prone).


(Also need to find a good and descriptive name for the flag we pass into 
`commitTx()` for this case.)



-Matthias



On 6/24/24 8:51 AM, Andrew Schofield wrote:

Hi Chris,
That works for me too. I slightly prefer an option on flush(), but what you 
suggested
works too.

Thanks,
Andrew


On 24 Jun 2024, at 15:14, Chris Egerton  wrote:

Hi Andrew,

I like a lot of what you said, but I still believe it's better to override
commitTransaction than flush. Users will already have to manually opt in to
ignoring errors encountered during transactions, and we can document
recommended usage (i.e., explicitly invoking flush() before invoking
commitTransaction(ignoreRecordErrors)) in the newly-introduced method. I
don't believe it's worth the increased cognitive load on users with
non-transactional producers to introduce an overloaded flush() variant.

Cheers,

Chris

On Mon, Jun 24, 2024 at 9:39 AM Andrew Schofield 
wrote:


Hi Alieh,
Thanks for driving this. Unfortunately, there are many parts of the API
which
are a bit unfortunate and it’s tricky to make small improvements that
don’t have
downsides.

I don’t like the idea of using a configuration because configuration is
often
outside the application and changing the behaviour of someone else’s
application
without understanding it is risky. Anything which embeds a transactional
producer
could have its behaviour changed unexpectedly.

It would be been much nicer if send() didn’t fail silently and change the
transaction
state. But, because it’s an asynchronous operation, I don’t really think
we can
just make it throw all exceptions, even though I really think that
`send()` is the
method with the problem here.

The contract of `flush()` is that it makes sure that all preceding sends
will have
completed, so it should be true that a well written application would be
able to
know which records were OK because of the Future returned
by the `send()` method. It should be able to determine whether it wants to
commit
the transaction even if some of the intended operations didn’t succeed.

What we don’t currently have is a way for the application to say to the
KafkaProducer
that it knows the outcome of sending the records and to confirm that it
wants to proceed.
Then it would not be necessary for `commitTransaction()` to throw an
exception to
report a historical error which the application might choose to ignore.

Having read the comments, I think the KIP is on the right lines focusing
on the `flush()`
method. My suggestion is that we introduce an option on `flush()` to be
used before
`commitTransaction()` for applications that want to be able to commit
transactions which
had known failed operations.

The code would be:

   producer.beginTransaction();

   future1 = producer.send(goodRecord1);
   future2 = producer.send(badRecord); // The future from this call will
complete exceptionally
   future3 = producer.send(good

Re: [DISCUSS] KIP-1059: Enable the Producer flush() method to clear the latest send() error

2024-06-24 Thread Andrew Schofield
Hi Chris,
That works for me too. I slightly prefer an option on flush(), but what you 
suggested
works too.

Thanks,
Andrew

> On 24 Jun 2024, at 15:14, Chris Egerton  wrote:
>
> Hi Andrew,
>
> I like a lot of what you said, but I still believe it's better to override
> commitTransaction than flush. Users will already have to manually opt in to
> ignoring errors encountered during transactions, and we can document
> recommended usage (i.e., explicitly invoking flush() before invoking
> commitTransaction(ignoreRecordErrors)) in the newly-introduced method. I
> don't believe it's worth the increased cognitive load on users with
> non-transactional producers to introduce an overloaded flush() variant.
>
> Cheers,
>
> Chris
>
> On Mon, Jun 24, 2024 at 9:39 AM Andrew Schofield 
> wrote:
>
>> Hi Alieh,
>> Thanks for driving this. Unfortunately, there are many parts of the API
>> which
>> are a bit unfortunate and it’s tricky to make small improvements that
>> don’t have
>> downsides.
>>
>> I don’t like the idea of using a configuration because configuration is
>> often
>> outside the application and changing the behaviour of someone else’s
>> application
>> without understanding it is risky. Anything which embeds a transactional
>> producer
>> could have its behaviour changed unexpectedly.
>>
>> It would be been much nicer if send() didn’t fail silently and change the
>> transaction
>> state. But, because it’s an asynchronous operation, I don’t really think
>> we can
>> just make it throw all exceptions, even though I really think that
>> `send()` is the
>> method with the problem here.
>>
>> The contract of `flush()` is that it makes sure that all preceding sends
>> will have
>> completed, so it should be true that a well written application would be
>> able to
>> know which records were OK because of the Future returned
>> by the `send()` method. It should be able to determine whether it wants to
>> commit
>> the transaction even if some of the intended operations didn’t succeed.
>>
>> What we don’t currently have is a way for the application to say to the
>> KafkaProducer
>> that it knows the outcome of sending the records and to confirm that it
>> wants to proceed.
>> Then it would not be necessary for `commitTransaction()` to throw an
>> exception to
>> report a historical error which the application might choose to ignore.
>>
>> Having read the comments, I think the KIP is on the right lines focusing
>> on the `flush()`
>> method. My suggestion is that we introduce an option on `flush()` to be
>> used before
>> `commitTransaction()` for applications that want to be able to commit
>> transactions which
>> had known failed operations.
>>
>> The code would be:
>>
>>   producer.beginTransaction();
>>
>>   future1 = producer.send(goodRecord1);
>>   future2 = producer.send(badRecord); // The future from this call will
>> complete exceptionally
>>   future3 = producer.send(goodRecord2);
>>
>>   producer.flush(FlushOption.TRANSACTION_READY);
>>
>>   // At this point, we know that all 3 futures are complete and the
>> transaction contains 2 records
>>   producer.commitTransaction();
>>
>> I wouldn’t deprecate `flush()` with no option. It just uses the default
>> option which behaves
>> like today.
>>
>> Why did I suggest an option on `flush()` rather than
>> `commitTransaction()`? Because with
>> `flush()`, it’s clear when the application is stating that it’s seen all
>> of the results from its
>> `send()` calls and it’s ready to proceed. If it has to rely on flushing
>> that occurs inside
>> `commitTransaction()`, I don’t see it’s as clear-cut.
>>
>> Thanks,
>> Andrew
>>
>>
>>
>>> On 24 Jun 2024, at 13:44, Alieh Saeedi 
>> wrote:
>>>
>>> Hi all,
>>> Thanks for the interesting discussion.
>>>
>>> I assume that now the main questions are as follows:
>>>
>>> 1. Do we need to transit the transcation to the error state for API
>>> exceptions?
>>> 2. Should we throw the API exception in `send()` instead of returning a
>>> future error?
>>> 3. If the answer to question (1) is NO and to question (2) is YES, do we
>>> need to change the current `flush` or `commitTnx` at all?
>>>
>>> Cheers,
>>> Alieh
>>>
>>> On Sat, Jun 22, 2024 at 3:21 AM Matthias J. Sax 
>> wrote:
>>>
 Hey Kirk,

 can you elaborate on a few points?

> Otherwise users would have to know to explicitly change their code to
 invoke flush().

 Why? If we would add an option to `flush(FlushOption)`, the existing
 `flush()` w/o any option will still be there, right? If we would really
 deprecate existing `flush()`, it would just mean that we would pass
 "default FlushOption" into an implicit flush (and yes, we would need to
 define what this would be).

 I think there is no clear winner (as pointed out in my last reply), and
 both `flush(FlushOption)` and `commitTx(CommitOption)` has advantages
 and drawbacks. Guess we need to just agree on which tradeoff we want to
 move forward with?

>>>

Re: [DISCUSS] KIP-1059: Enable the Producer flush() method to clear the latest send() error

2024-06-24 Thread Chris Egerton
Hi Andrew,

I like a lot of what you said, but I still believe it's better to override
commitTransaction than flush. Users will already have to manually opt in to
ignoring errors encountered during transactions, and we can document
recommended usage (i.e., explicitly invoking flush() before invoking
commitTransaction(ignoreRecordErrors)) in the newly-introduced method. I
don't believe it's worth the increased cognitive load on users with
non-transactional producers to introduce an overloaded flush() variant.

Cheers,

Chris

On Mon, Jun 24, 2024 at 9:39 AM Andrew Schofield 
wrote:

> Hi Alieh,
> Thanks for driving this. Unfortunately, there are many parts of the API
> which
> are a bit unfortunate and it’s tricky to make small improvements that
> don’t have
> downsides.
>
> I don’t like the idea of using a configuration because configuration is
> often
> outside the application and changing the behaviour of someone else’s
> application
> without understanding it is risky. Anything which embeds a transactional
> producer
> could have its behaviour changed unexpectedly.
>
> It would be been much nicer if send() didn’t fail silently and change the
> transaction
> state. But, because it’s an asynchronous operation, I don’t really think
> we can
> just make it throw all exceptions, even though I really think that
> `send()` is the
> method with the problem here.
>
> The contract of `flush()` is that it makes sure that all preceding sends
> will have
> completed, so it should be true that a well written application would be
> able to
> know which records were OK because of the Future returned
> by the `send()` method. It should be able to determine whether it wants to
> commit
> the transaction even if some of the intended operations didn’t succeed.
>
> What we don’t currently have is a way for the application to say to the
> KafkaProducer
> that it knows the outcome of sending the records and to confirm that it
> wants to proceed.
> Then it would not be necessary for `commitTransaction()` to throw an
> exception to
> report a historical error which the application might choose to ignore.
>
> Having read the comments, I think the KIP is on the right lines focusing
> on the `flush()`
> method. My suggestion is that we introduce an option on `flush()` to be
> used before
> `commitTransaction()` for applications that want to be able to commit
> transactions which
> had known failed operations.
>
> The code would be:
>
>producer.beginTransaction();
>
>future1 = producer.send(goodRecord1);
>future2 = producer.send(badRecord); // The future from this call will
> complete exceptionally
>future3 = producer.send(goodRecord2);
>
>producer.flush(FlushOption.TRANSACTION_READY);
>
>// At this point, we know that all 3 futures are complete and the
> transaction contains 2 records
>producer.commitTransaction();
>
> I wouldn’t deprecate `flush()` with no option. It just uses the default
> option which behaves
> like today.
>
> Why did I suggest an option on `flush()` rather than
> `commitTransaction()`? Because with
> `flush()`, it’s clear when the application is stating that it’s seen all
> of the results from its
> `send()` calls and it’s ready to proceed. If it has to rely on flushing
> that occurs inside
> `commitTransaction()`, I don’t see it’s as clear-cut.
>
> Thanks,
> Andrew
>
>
>
> > On 24 Jun 2024, at 13:44, Alieh Saeedi 
> wrote:
> >
> > Hi all,
> > Thanks for the interesting discussion.
> >
> > I assume that now the main questions are as follows:
> >
> > 1. Do we need to transit the transcation to the error state for API
> > exceptions?
> > 2. Should we throw the API exception in `send()` instead of returning a
> > future error?
> > 3. If the answer to question (1) is NO and to question (2) is YES, do we
> > need to change the current `flush` or `commitTnx` at all?
> >
> > Cheers,
> > Alieh
> >
> > On Sat, Jun 22, 2024 at 3:21 AM Matthias J. Sax 
> wrote:
> >
> >> Hey Kirk,
> >>
> >> can you elaborate on a few points?
> >>
> >>> Otherwise users would have to know to explicitly change their code to
> >> invoke flush().
> >>
> >> Why? If we would add an option to `flush(FlushOption)`, the existing
> >> `flush()` w/o any option will still be there, right? If we would really
> >> deprecate existing `flush()`, it would just mean that we would pass
> >> "default FlushOption" into an implicit flush (and yes, we would need to
> >> define what this would be).
> >>
> >> I think there is no clear winner (as pointed out in my last reply), and
> >> both `flush(FlushOption)` and `commitTx(CommitOption)` has advantages
> >> and drawbacks. Guess we need to just agree on which tradeoff we want to
> >> move forward with?
> >>
> >>
> >> Not sure if your database example is a 1:1 fit? I think, the better
> >> comparison would be:
> >>
> >> BEGIN TX;
> >> INSERT INTO foo VALUES (’a’);
> >> INSERT INTO foo VALUES (’b’);
> >> INSERT INTO foo VALUES (’c’);
> >> INSERT INTO foo VALUES (’not sure’);
> >>
> >> For

Re: [DISCUSS] KIP-1059: Enable the Producer flush() method to clear the latest send() error

2024-06-24 Thread Andrew Schofield
Hi Alieh,
Thanks for driving this. Unfortunately, there are many parts of the API which
are a bit unfortunate and it’s tricky to make small improvements that don’t have
downsides.

I don’t like the idea of using a configuration because configuration is often
outside the application and changing the behaviour of someone else’s application
without understanding it is risky. Anything which embeds a transactional 
producer
could have its behaviour changed unexpectedly.

It would be been much nicer if send() didn’t fail silently and change the 
transaction
state. But, because it’s an asynchronous operation, I don’t really think we can
just make it throw all exceptions, even though I really think that `send()` is 
the
method with the problem here.

The contract of `flush()` is that it makes sure that all preceding sends will 
have
completed, so it should be true that a well written application would be able to
know which records were OK because of the Future returned
by the `send()` method. It should be able to determine whether it wants to 
commit
the transaction even if some of the intended operations didn’t succeed.

What we don’t currently have is a way for the application to say to the 
KafkaProducer
that it knows the outcome of sending the records and to confirm that it wants 
to proceed.
Then it would not be necessary for `commitTransaction()` to throw an exception 
to
report a historical error which the application might choose to ignore.

Having read the comments, I think the KIP is on the right lines focusing on the 
`flush()`
method. My suggestion is that we introduce an option on `flush()` to be used 
before
`commitTransaction()` for applications that want to be able to commit 
transactions which
had known failed operations.

The code would be:

   producer.beginTransaction();

   future1 = producer.send(goodRecord1);
   future2 = producer.send(badRecord); // The future from this call will 
complete exceptionally
   future3 = producer.send(goodRecord2);

   producer.flush(FlushOption.TRANSACTION_READY);

   // At this point, we know that all 3 futures are complete and the 
transaction contains 2 records
   producer.commitTransaction();

I wouldn’t deprecate `flush()` with no option. It just uses the default option 
which behaves
like today.

Why did I suggest an option on `flush()` rather than `commitTransaction()`? 
Because with
`flush()`, it’s clear when the application is stating that it’s seen all of the 
results from its
`send()` calls and it’s ready to proceed. If it has to rely on flushing that 
occurs inside
`commitTransaction()`, I don’t see it’s as clear-cut.

Thanks,
Andrew



> On 24 Jun 2024, at 13:44, Alieh Saeedi  wrote:
>
> Hi all,
> Thanks for the interesting discussion.
>
> I assume that now the main questions are as follows:
>
> 1. Do we need to transit the transcation to the error state for API
> exceptions?
> 2. Should we throw the API exception in `send()` instead of returning a
> future error?
> 3. If the answer to question (1) is NO and to question (2) is YES, do we
> need to change the current `flush` or `commitTnx` at all?
>
> Cheers,
> Alieh
>
> On Sat, Jun 22, 2024 at 3:21 AM Matthias J. Sax  wrote:
>
>> Hey Kirk,
>>
>> can you elaborate on a few points?
>>
>>> Otherwise users would have to know to explicitly change their code to
>> invoke flush().
>>
>> Why? If we would add an option to `flush(FlushOption)`, the existing
>> `flush()` w/o any option will still be there, right? If we would really
>> deprecate existing `flush()`, it would just mean that we would pass
>> "default FlushOption" into an implicit flush (and yes, we would need to
>> define what this would be).
>>
>> I think there is no clear winner (as pointed out in my last reply), and
>> both `flush(FlushOption)` and `commitTx(CommitOption)` has advantages
>> and drawbacks. Guess we need to just agree on which tradeoff we want to
>> move forward with?
>>
>>
>> Not sure if your database example is a 1:1 fit? I think, the better
>> comparison would be:
>>
>> BEGIN TX;
>> INSERT INTO foo VALUES (’a’);
>> INSERT INTO foo VALUES (’b’);
>> INSERT INTO foo VALUES (’c’);
>> INSERT INTO foo VALUES (’not sure’);
>>
>> For this case, the full TX would roll back, right? I still think that
>> allowing users to just skip over the last error, and continue the TX
>> would be ok. In the end, we provide a programmatic API, and not a
>> declarative one as SQL. Of course, default behavior would still be to
>> put the producer into error state, and the user would need to call
>> `abortTransaction()` to move forward.
>>
>>
>> -Matthias
>>
>> On 6/21/24 5:26 PM, Kirk True wrote:
>>> Hi Matthias,
>>>
 On Jun 21, 2024, at 12:28 PM, Matthias J. Sax  wrote:

 If we want to limit it to `RecordTooLargeException` throwing from
>> `send()` directly make sense. Thanks for calling it out.

 It's still a question of backward compatibility? `send()` does throw
>> exceptions already, including generic `KafkaException`. Not sure

Re: [DISCUSS] KIP-1059: Enable the Producer flush() method to clear the latest send() error

2024-06-24 Thread Alieh Saeedi
Hi all,
Thanks for the interesting discussion.

I assume that now the main questions are as follows:

1. Do we need to transit the transcation to the error state for API
exceptions?
2. Should we throw the API exception in `send()` instead of returning a
future error?
3. If the answer to question (1) is NO and to question (2) is YES, do we
need to change the current `flush` or `commitTnx` at all?

Cheers,
Alieh

On Sat, Jun 22, 2024 at 3:21 AM Matthias J. Sax  wrote:

> Hey Kirk,
>
> can you elaborate on a few points?
>
> > Otherwise users would have to know to explicitly change their code to
> invoke flush().
>
> Why? If we would add an option to `flush(FlushOption)`, the existing
> `flush()` w/o any option will still be there, right? If we would really
> deprecate existing `flush()`, it would just mean that we would pass
> "default FlushOption" into an implicit flush (and yes, we would need to
> define what this would be).
>
> I think there is no clear winner (as pointed out in my last reply), and
> both `flush(FlushOption)` and `commitTx(CommitOption)` has advantages
> and drawbacks. Guess we need to just agree on which tradeoff we want to
> move forward with?
>
>
> Not sure if your database example is a 1:1 fit? I think, the better
> comparison would be:
>
> BEGIN TX;
> INSERT INTO foo VALUES (’a’);
> INSERT INTO foo VALUES (’b’);
> INSERT INTO foo VALUES (’c’);
> INSERT INTO foo VALUES (’not sure’);
>
> For this case, the full TX would roll back, right? I still think that
> allowing users to just skip over the last error, and continue the TX
> would be ok. In the end, we provide a programmatic API, and not a
> declarative one as SQL. Of course, default behavior would still be to
> put the producer into error state, and the user would need to call
> `abortTransaction()` to move forward.
>
>
> -Matthias
>
> On 6/21/24 5:26 PM, Kirk True wrote:
> > Hi Matthias,
> >
> >> On Jun 21, 2024, at 12:28 PM, Matthias J. Sax  wrote:
> >>
> >> If we want to limit it to `RecordTooLargeException` throwing from
> `send()` directly make sense. Thanks for calling it out.
> >>
> >> It's still a question of backward compatibility? `send()` does throw
> exceptions already, including generic `KafkaException`. Not sure if this
> helps with backward compatibility? Could we just add a new exception type
> (which is a child of `KafkaException`)?
> >>
> >> The Producer JavaDocs are not totally explicit about it IMHO.
> >>
> >> I think we could expect that some generic error handling path gets
> executed. For the TX-case, I would assume that a TX would be aborted if
> `send()` throws or that the producer would be `closed()`. Overall this
> might be safe?
> >>
> >>> It would be a little less flexible
>  though, since (as you note) it would still be impossible to commit
>  transactions after errors have been reported from brokers.
> >>
> >> KS would still need a way to clear the error state of the producer. We
> could catch a `RecordTooLargeException` from `send()`, call the handler and
> let it decide what to do next. But if it does return `CONTINUE` to swallow
> the error and drop the poison pill record on the floor, we would want to
> move forward and commit the transaction.
> >>
> >> But the question is: if we cannot add a record to the tx, does the
> producer need to go into error state? In the end, we did throw and inform
> the app that the record was _not_ added, and it's up to the app to decide
> what to do next?
> >
> > That’s an excellent question…
> >
> > Imagine the user’s application is writing information to a database
> instead of Kafka. If there’s a table with a CHAR(1) column and this SQL
> statement was attempted, what should happen?
> >
> >  INSERT INTO foo VALUES (’not sure’);
> >
> > Yes, that DML would fail, sure, but would the user expect that the
> connection used by database library would get stuck in some kind of error
> state? A user would be able catch the error and either continue or abort,
> based on their business rules.
> >
> > So I agree with what I believe you’re implying: we shouldn’t poison the
> Producer/TransactionManager on certain types of application-level errors in
> send().
> >
> > Kirk
> >
> >> If we report the error only via the `Callback` it's a different story,
> because the contract for this case is clearly specified on the JavaDocs:
> >>
> >>> When used as part of a transaction, it is not necessary to define a
> callback or check the result of the future
> >>> in order to detect errors from send. If any of the send
> calls failed with an irrecoverable error,
> >>> the final {@link #commitTransaction()} call will fail and throw the
> exception from the last failed send. When
> >>> this happens, your application should call {@link #abortTransaction()}
> to reset the state and continue to send
> >>> data.
> >>
> >>
> >>
> >> -Matthias
> >>
> >>
> >> On 6/21/24 11:42 AM, Chris Egerton wrote:
> >>> Hi Artem,
> >>> I think it'd make sense to throw directly from send whenever pos

Re: [DISCUSS] KIP-1059: Enable the Producer flush() method to clear the latest send() error

2024-06-21 Thread Matthias J. Sax

Hey Kirk,

can you elaborate on a few points?


Otherwise users would have to know to explicitly change their code to invoke 
flush().


Why? If we would add an option to `flush(FlushOption)`, the existing 
`flush()` w/o any option will still be there, right? If we would really 
deprecate existing `flush()`, it would just mean that we would pass 
"default FlushOption" into an implicit flush (and yes, we would need to 
define what this would be).


I think there is no clear winner (as pointed out in my last reply), and 
both `flush(FlushOption)` and `commitTx(CommitOption)` has advantages 
and drawbacks. Guess we need to just agree on which tradeoff we want to 
move forward with?



Not sure if your database example is a 1:1 fit? I think, the better 
comparison would be:


BEGIN TX;
INSERT INTO foo VALUES (’a’);
INSERT INTO foo VALUES (’b’);
INSERT INTO foo VALUES (’c’);
INSERT INTO foo VALUES (’not sure’);

For this case, the full TX would roll back, right? I still think that 
allowing users to just skip over the last error, and continue the TX 
would be ok. In the end, we provide a programmatic API, and not a 
declarative one as SQL. Of course, default behavior would still be to 
put the producer into error state, and the user would need to call 
`abortTransaction()` to move forward.



-Matthias

On 6/21/24 5:26 PM, Kirk True wrote:

Hi Matthias,


On Jun 21, 2024, at 12:28 PM, Matthias J. Sax  wrote:

If we want to limit it to `RecordTooLargeException` throwing from `send()` 
directly make sense. Thanks for calling it out.

It's still a question of backward compatibility? `send()` does throw exceptions 
already, including generic `KafkaException`. Not sure if this helps with 
backward compatibility? Could we just add a new exception type (which is a 
child of `KafkaException`)?

The Producer JavaDocs are not totally explicit about it IMHO.

I think we could expect that some generic error handling path gets executed. 
For the TX-case, I would assume that a TX would be aborted if `send()` throws 
or that the producer would be `closed()`. Overall this might be safe?


It would be a little less flexible

though, since (as you note) it would still be impossible to commit
transactions after errors have been reported from brokers.


KS would still need a way to clear the error state of the producer. We could 
catch a `RecordTooLargeException` from `send()`, call the handler and let it 
decide what to do next. But if it does return `CONTINUE` to swallow the error 
and drop the poison pill record on the floor, we would want to move forward and 
commit the transaction.

But the question is: if we cannot add a record to the tx, does the producer 
need to go into error state? In the end, we did throw and inform the app that 
the record was _not_ added, and it's up to the app to decide what to do next?


That’s an excellent question…

Imagine the user’s application is writing information to a database instead of 
Kafka. If there’s a table with a CHAR(1) column and this SQL statement was 
attempted, what should happen?

 INSERT INTO foo VALUES (’not sure’);

Yes, that DML would fail, sure, but would the user expect that the connection 
used by database library would get stuck in some kind of error state? A user 
would be able catch the error and either continue or abort, based on their 
business rules.

So I agree with what I believe you’re implying: we shouldn’t poison the 
Producer/TransactionManager on certain types of application-level errors in 
send().

Kirk


If we report the error only via the `Callback` it's a different story, because 
the contract for this case is clearly specified on the JavaDocs:


When used as part of a transaction, it is not necessary to define a callback or 
check the result of the future
in order to detect errors from send. If any of the send calls 
failed with an irrecoverable error,
the final {@link #commitTransaction()} call will fail and throw the exception 
from the last failed send. When
this happens, your application should call {@link #abortTransaction()} to reset 
the state and continue to send
data.




-Matthias


On 6/21/24 11:42 AM, Chris Egerton wrote:

Hi Artem,
I think it'd make sense to throw directly from send whenever possible,
instead of returning an already-completed future. I didn't do that in my
bug fix to try to be conservative about breaking changes but this seems to
have caused its own set of headaches. It would be a little less flexible
though, since (as you note) it would still be impossible to commit
transactions after errors have been reported from brokers.
I'll leave it up to the Kafka Streams folks to decide if that flexibility
is required. If it is, then users could explicitly call flush() before
committing (and ignoring errors for) or aborting a transaction, if they
want to implement fine-grained error handling logic such as allowing errors
for a subset of topics to be ignored.
Hi Matthias,
Most of the time you're right and we can't throw from se

Re: [DISCUSS] KIP-1059: Enable the Producer flush() method to clear the latest send() error

2024-06-21 Thread Kirk True
Hi Matthias,

> On Jun 21, 2024, at 12:28 PM, Matthias J. Sax  wrote:
> 
> If we want to limit it to `RecordTooLargeException` throwing from `send()` 
> directly make sense. Thanks for calling it out.
> 
> It's still a question of backward compatibility? `send()` does throw 
> exceptions already, including generic `KafkaException`. Not sure if this 
> helps with backward compatibility? Could we just add a new exception type 
> (which is a child of `KafkaException`)?
> 
> The Producer JavaDocs are not totally explicit about it IMHO.
> 
> I think we could expect that some generic error handling path gets executed. 
> For the TX-case, I would assume that a TX would be aborted if `send()` throws 
> or that the producer would be `closed()`. Overall this might be safe?
> 
>> It would be a little less flexible
>>> though, since (as you note) it would still be impossible to commit
>>> transactions after errors have been reported from brokers.
> 
> KS would still need a way to clear the error state of the producer. We could 
> catch a `RecordTooLargeException` from `send()`, call the handler and let it 
> decide what to do next. But if it does return `CONTINUE` to swallow the error 
> and drop the poison pill record on the floor, we would want to move forward 
> and commit the transaction.
> 
> But the question is: if we cannot add a record to the tx, does the producer 
> need to go into error state? In the end, we did throw and inform the app that 
> the record was _not_ added, and it's up to the app to decide what to do next?

That’s an excellent question…

Imagine the user’s application is writing information to a database instead of 
Kafka. If there’s a table with a CHAR(1) column and this SQL statement was 
attempted, what should happen?

INSERT INTO foo VALUES (’not sure’);

Yes, that DML would fail, sure, but would the user expect that the connection 
used by database library would get stuck in some kind of error state? A user 
would be able catch the error and either continue or abort, based on their 
business rules.

So I agree with what I believe you’re implying: we shouldn’t poison the 
Producer/TransactionManager on certain types of application-level errors in 
send().

Kirk

> If we report the error only via the `Callback` it's a different story, 
> because the contract for this case is clearly specified on the JavaDocs:
> 
>> When used as part of a transaction, it is not necessary to define a callback 
>> or check the result of the future
>> in order to detect errors from send. If any of the send calls 
>> failed with an irrecoverable error,
>> the final {@link #commitTransaction()} call will fail and throw the 
>> exception from the last failed send. When
>> this happens, your application should call {@link #abortTransaction()} to 
>> reset the state and continue to send
>> data.
> 
> 
> 
> -Matthias
> 
> 
> On 6/21/24 11:42 AM, Chris Egerton wrote:
>> Hi Artem,
>> I think it'd make sense to throw directly from send whenever possible,
>> instead of returning an already-completed future. I didn't do that in my
>> bug fix to try to be conservative about breaking changes but this seems to
>> have caused its own set of headaches. It would be a little less flexible
>> though, since (as you note) it would still be impossible to commit
>> transactions after errors have been reported from brokers.
>> I'll leave it up to the Kafka Streams folks to decide if that flexibility
>> is required. If it is, then users could explicitly call flush() before
>> committing (and ignoring errors for) or aborting a transaction, if they
>> want to implement fine-grained error handling logic such as allowing errors
>> for a subset of topics to be ignored.
>> Hi Matthias,
>> Most of the time you're right and we can't throw from send(); however, in
>> this case (client-side record-too-large exception), the error is actually
>> noticed by the producer before send() returns, so it should be possible to
>> throw directly.
>> Cheers,
>> Chris
>> On Fri, Jun 21, 2024, 14:25 Matthias J. Sax  wrote:
>>> Not sure if we can change send and make it throw, given that send() is
>>> async? That is why users can register a `Callback` to begin with, right?
>>> 
>>> And Alieh's point about backward compatibility is also a fair concern.
>>> 
>>> 
 Actually, this would potentially be even
 worse than the original buggy behavior because the bug was that we
>>> ignored
 errors that happened in the "send()" method itself, not necessarily the
 ones that we got from the broker.
>>> 
>>> My understanding was that `commitTx(swallowError)` would only swallow
>>> `send()` errors, not errors about the actually commit. I agree that it
>>> would be very bad to swallow errors about the actual tx commit...
>>> 
>>> It's a fair question if this might be too subtle; to make it explicit,
>>> we could use `CommitOpions#ignorePendingSendErors()` [working name] to
>>> make it clear.
>>> 
>>> 
>>> If we think it's too subtle to change commit to sw

Re: [DISCUSS] KIP-1059: Enable the Producer flush() method to clear the latest send() error

2024-06-21 Thread Kirk True
Hi all,

The JavaDoc for Producer.flush() states:

Applications don't need to call this method for transactional producers, since 
the commitTransaction() will flush all buffered records before performing the 
commit. This ensures that all the send() calls made since the previous 
beginTransaction() are completed before the commit.

Because the documentation states that invoking flush() isn’t necessary, if we 
added a flag to flush(), wouldn’t we need to add that flag to 
commitTransaction() too? Otherwise users would have to know to explicitly 
change their code to invoke flush(). Also, flush() is used outside of the 
transactional Producer use case, adding a transaction-specific flag to that 
method is a bit confusing. 

As Chris mentioned, it makes intuitive sense to introduce behavior changes as 
close as possible to where it’s relevant, i.e. commitTransaction(). A 
“CommitOptions” class would follow the pattern used in the Admin client.

Thanks,
Kirk

> On Jun 21, 2024, at 12:28 PM, Matthias J. Sax  wrote:
> 
> If we want to limit it to `RecordTooLargeException` throwing from `send()` 
> directly make sense. Thanks for calling it out.
> 
> It's still a question of backward compatibility? `send()` does throw 
> exceptions already, including generic `KafkaException`. Not sure if this 
> helps with backward compatibility? Could we just add a new exception type 
> (which is a child of `KafkaException`)?
> 
> The Producer JavaDocs are not totally explicit about it IMHO.
> 
> I think we could expect that some generic error handling path gets executed. 
> For the TX-case, I would assume that a TX would be aborted if `send()` throws 
> or that the producer would be `closed()`. Overall this might be safe?
> 
>> It would be a little less flexible
>>> though, since (as you note) it would still be impossible to commit
>>> transactions after errors have been reported from brokers.
> 
> KS would still need a way to clear the error state of the producer. We could 
> catch a `RecordTooLargeException` from `send()`, call the handler and let it 
> decide what to do next. But if it does return `CONTINUE` to swallow the error 
> and drop the poison pill record on the floor, we would want to move forward 
> and commit the transaction.
> 
> But the question is: if we cannot add a record to the tx, does the producer 
> need to go into error state? In the end, we did throw and inform the app that 
> the record was _not_ added, and it's up to the app to decide what to do next?
> 
> If we report the error only via the `Callback` it's a different story, 
> because the contract for this case is clearly specified on the JavaDocs:
> 
>> When used as part of a transaction, it is not necessary to define a callback 
>> or check the result of the future
>> in order to detect errors from send. If any of the send calls 
>> failed with an irrecoverable error,
>> the final {@link #commitTransaction()} call will fail and throw the 
>> exception from the last failed send. When
>> this happens, your application should call {@link #abortTransaction()} to 
>> reset the state and continue to send
>> data.
> 
> 
> 
> -Matthias
> 
> 
> On 6/21/24 11:42 AM, Chris Egerton wrote:
>> Hi Artem,
>> I think it'd make sense to throw directly from send whenever possible,
>> instead of returning an already-completed future. I didn't do that in my
>> bug fix to try to be conservative about breaking changes but this seems to
>> have caused its own set of headaches. It would be a little less flexible
>> though, since (as you note) it would still be impossible to commit
>> transactions after errors have been reported from brokers.
>> I'll leave it up to the Kafka Streams folks to decide if that flexibility
>> is required. If it is, then users could explicitly call flush() before
>> committing (and ignoring errors for) or aborting a transaction, if they
>> want to implement fine-grained error handling logic such as allowing errors
>> for a subset of topics to be ignored.
>> Hi Matthias,
>> Most of the time you're right and we can't throw from send(); however, in
>> this case (client-side record-too-large exception), the error is actually
>> noticed by the producer before send() returns, so it should be possible to
>> throw directly.
>> Cheers,
>> Chris
>> On Fri, Jun 21, 2024, 14:25 Matthias J. Sax  wrote:
>>> Not sure if we can change send and make it throw, given that send() is
>>> async? That is why users can register a `Callback` to begin with, right?
>>> 
>>> And Alieh's point about backward compatibility is also a fair concern.
>>> 
>>> 
 Actually, this would potentially be even
 worse than the original buggy behavior because the bug was that we
>>> ignored
 errors that happened in the "send()" method itself, not necessarily the
 ones that we got from the broker.
>>> 
>>> My understanding was that `commitTx(swallowError)` would only swallow
>>> `send()` errors, not errors about the actually commit. I agree that it
>>> would be very bad to 

Re: [DISCUSS] KIP-1059: Enable the Producer flush() method to clear the latest send() error

2024-06-21 Thread Matthias J. Sax
If we want to limit it to `RecordTooLargeException` throwing from 
`send()` directly make sense. Thanks for calling it out.


It's still a question of backward compatibility? `send()` does throw 
exceptions already, including generic `KafkaException`. Not sure if this 
helps with backward compatibility? Could we just add a new exception 
type (which is a child of `KafkaException`)?


The Producer JavaDocs are not totally explicit about it IMHO.

I think we could expect that some generic error handling path gets 
executed. For the TX-case, I would assume that a TX would be aborted if 
`send()` throws or that the producer would be `closed()`. Overall this 
might be safe?



It would be a little less flexible

though, since (as you note) it would still be impossible to commit
transactions after errors have been reported from brokers.


KS would still need a way to clear the error state of the producer. We 
could catch a `RecordTooLargeException` from `send()`, call the handler 
and let it decide what to do next. But if it does return `CONTINUE` to 
swallow the error and drop the poison pill record on the floor, we would 
want to move forward and commit the transaction.


But the question is: if we cannot add a record to the tx, does the 
producer need to go into error state? In the end, we did throw and 
inform the app that the record was _not_ added, and it's up to the app 
to decide what to do next?


If we report the error only via the `Callback` it's a different story, 
because the contract for this case is clearly specified on the JavaDocs:



When used as part of a transaction, it is not necessary to define a callback or 
check the result of the future
in order to detect errors from send. If any of the send calls 
failed with an irrecoverable error,
the final {@link #commitTransaction()} call will fail and throw the exception 
from the last failed send. When
this happens, your application should call {@link #abortTransaction()} to reset 
the state and continue to send
data.




-Matthias


On 6/21/24 11:42 AM, Chris Egerton wrote:

Hi Artem,

I think it'd make sense to throw directly from send whenever possible,
instead of returning an already-completed future. I didn't do that in my
bug fix to try to be conservative about breaking changes but this seems to
have caused its own set of headaches. It would be a little less flexible
though, since (as you note) it would still be impossible to commit
transactions after errors have been reported from brokers.

I'll leave it up to the Kafka Streams folks to decide if that flexibility
is required. If it is, then users could explicitly call flush() before
committing (and ignoring errors for) or aborting a transaction, if they
want to implement fine-grained error handling logic such as allowing errors
for a subset of topics to be ignored.

Hi Matthias,

Most of the time you're right and we can't throw from send(); however, in
this case (client-side record-too-large exception), the error is actually
noticed by the producer before send() returns, so it should be possible to
throw directly.

Cheers,

Chris

On Fri, Jun 21, 2024, 14:25 Matthias J. Sax  wrote:


Not sure if we can change send and make it throw, given that send() is
async? That is why users can register a `Callback` to begin with, right?

And Alieh's point about backward compatibility is also a fair concern.



Actually, this would potentially be even
worse than the original buggy behavior because the bug was that we

ignored

errors that happened in the "send()" method itself, not necessarily the
ones that we got from the broker.


My understanding was that `commitTx(swallowError)` would only swallow
`send()` errors, not errors about the actually commit. I agree that it
would be very bad to swallow errors about the actual tx commit...

It's a fair question if this might be too subtle; to make it explicit,
we could use `CommitOpions#ignorePendingSendErors()` [working name] to
make it clear.


If we think it's too subtle to change commit to swallow send() errors,
maybe going with `flush(FlushOptions)` would be clearer (and we can use
`FlushOption#swallowSendErrorsForTransactions()` [working name] to be
explicitly that the `FlushOption` for now has only an effect for TX).


Thoughts?


-Matthias



On 6/21/24 4:10 AM, Alieh Saeedi wrote:

Hi all,


It is very exciting to see all the experts here raising very good points.

As we go further, we see more and more options to improve our solution,
which makes concluding and updating the KIP impossible.


The main suggestions so far are:

1. `flush` with `flushOptions` as input parameter

2. `commitTx` with `commitOptions` as input parameter

3. `send` must throw the exception


My concern about the 3rd suggestion:

1. Does the change cause any issue with backward compatibility?

2. The `send (bad record)` already transits the transaction to the error
state. No user, including Streams is able to transit the transaction back
from the error state. Do you mean we 

Re: [DISCUSS] KIP-1059: Enable the Producer flush() method to clear the latest send() error

2024-06-21 Thread Chris Egerton
Hi Artem,

I think it'd make sense to throw directly from send whenever possible,
instead of returning an already-completed future. I didn't do that in my
bug fix to try to be conservative about breaking changes but this seems to
have caused its own set of headaches. It would be a little less flexible
though, since (as you note) it would still be impossible to commit
transactions after errors have been reported from brokers.

I'll leave it up to the Kafka Streams folks to decide if that flexibility
is required. If it is, then users could explicitly call flush() before
committing (and ignoring errors for) or aborting a transaction, if they
want to implement fine-grained error handling logic such as allowing errors
for a subset of topics to be ignored.

Hi Matthias,

Most of the time you're right and we can't throw from send(); however, in
this case (client-side record-too-large exception), the error is actually
noticed by the producer before send() returns, so it should be possible to
throw directly.

Cheers,

Chris

On Fri, Jun 21, 2024, 14:25 Matthias J. Sax  wrote:

> Not sure if we can change send and make it throw, given that send() is
> async? That is why users can register a `Callback` to begin with, right?
>
> And Alieh's point about backward compatibility is also a fair concern.
>
>
> > Actually, this would potentially be even
> > worse than the original buggy behavior because the bug was that we
> ignored
> > errors that happened in the "send()" method itself, not necessarily the
> > ones that we got from the broker.
>
> My understanding was that `commitTx(swallowError)` would only swallow
> `send()` errors, not errors about the actually commit. I agree that it
> would be very bad to swallow errors about the actual tx commit...
>
> It's a fair question if this might be too subtle; to make it explicit,
> we could use `CommitOpions#ignorePendingSendErors()` [working name] to
> make it clear.
>
>
> If we think it's too subtle to change commit to swallow send() errors,
> maybe going with `flush(FlushOptions)` would be clearer (and we can use
> `FlushOption#swallowSendErrorsForTransactions()` [working name] to be
> explicitly that the `FlushOption` for now has only an effect for TX).
>
>
> Thoughts?
>
>
> -Matthias
>
>
>
> On 6/21/24 4:10 AM, Alieh Saeedi wrote:
> > Hi all,
> >
> >
> > It is very exciting to see all the experts here raising very good points.
> >
> > As we go further, we see more and more options to improve our solution,
> > which makes concluding and updating the KIP impossible.
> >
> >
> > The main suggestions so far are:
> >
> > 1. `flush` with `flushOptions` as input parameter
> >
> > 2. `commitTx` with `commitOptions` as input parameter
> >
> > 3. `send` must throw the exception
> >
> >
> > My concern about the 3rd suggestion:
> >
> > 1. Does the change cause any issue with backward compatibility?
> >
> > 2. The `send (bad record)` already transits the transaction to the error
> > state. No user, including Streams is able to transit the transaction back
> > from the error state. Do you mean we remove the
> > `maybeTransitionToErrorState(e)` from here
> > <
> https://github.com/apache/kafka/blob/9b5b434e2a6b2d5290ea403fc02859b1c523d8aa/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java#L1112
> >
> > as well?
> >
> > Cheers,
> > Alieh
> >
> >
> > On Fri, Jun 21, 2024 at 8:45 AM Andrew Schofield <
> andrew_schofi...@live.com>
> > wrote:
> >
> >> Hi Artem,
> >> I think you make a good point which is worth further consideration. If
> >> any of the existing methods is really ripe for a change here, it’s the
> >> send() that actually caused the problem. If that can be fixed so there
> are
> >> no situations in which a lurking error breaks a transaction, that might
> be
> >> the best.
> >>
> >> Thanks,
> >> Andrew
> >>
> >>> On 21 Jun 2024, at 01:51, Artem Livshits  .INVALID>
> >> wrote:
> >>>
>  I thought we still wait for requests (and their errors) to come in and
> >>> could handle fatal errors appropriately.
> >>>
> >>> We do wait for requests, but my understanding is that when
> >>> commitTransaction("ignore send errors") we want to ignore errors.  So
> if
> >> we
> >>> do
> >>>
> >>> 1. send
> >>> 2. commitTransaction("ignore send errors")
> >>>
> >>> the commit will succeed.  You can look at the example in
> >>> https://issues.apache.org/jira/browse/KAFKA-9279 and just substitute
> >>> commitTransaction with commitTransaction("ignore send errors") and we
> get
> >>> the buggy behavior back :-).  Actually, this would potentially be even
> >>> worse than the original buggy behavior because the bug was that we
> >> ignored
> >>> errors that happened in the "send()" method itself, not necessarily the
> >>> ones that we got from the broker.
> >>>
> >>> Actually, looking at https://github.com/apache/kafka/pull/11508/files,
> >>> wouldn't a better solution be to just throw the error from the "send"
> >>> method itself, rather than trying to set it to be thrown d

Re: [DISCUSS] KIP-1059: Enable the Producer flush() method to clear the latest send() error

2024-06-21 Thread Matthias J. Sax
Not sure if we can change send and make it throw, given that send() is 
async? That is why users can register a `Callback` to begin with, right?


And Alieh's point about backward compatibility is also a fair concern.



Actually, this would potentially be even
worse than the original buggy behavior because the bug was that we ignored
errors that happened in the "send()" method itself, not necessarily the
ones that we got from the broker.


My understanding was that `commitTx(swallowError)` would only swallow 
`send()` errors, not errors about the actually commit. I agree that it 
would be very bad to swallow errors about the actual tx commit...


It's a fair question if this might be too subtle; to make it explicit, 
we could use `CommitOpions#ignorePendingSendErors()` [working name] to 
make it clear.



If we think it's too subtle to change commit to swallow send() errors, 
maybe going with `flush(FlushOptions)` would be clearer (and we can use 
`FlushOption#swallowSendErrorsForTransactions()` [working name] to be 
explicitly that the `FlushOption` for now has only an effect for TX).



Thoughts?


-Matthias



On 6/21/24 4:10 AM, Alieh Saeedi wrote:

Hi all,


It is very exciting to see all the experts here raising very good points.

As we go further, we see more and more options to improve our solution,
which makes concluding and updating the KIP impossible.


The main suggestions so far are:

1. `flush` with `flushOptions` as input parameter

2. `commitTx` with `commitOptions` as input parameter

3. `send` must throw the exception


My concern about the 3rd suggestion:

1. Does the change cause any issue with backward compatibility?

2. The `send (bad record)` already transits the transaction to the error
state. No user, including Streams is able to transit the transaction back
from the error state. Do you mean we remove the
`maybeTransitionToErrorState(e)` from here

as well?

Cheers,
Alieh


On Fri, Jun 21, 2024 at 8:45 AM Andrew Schofield 
wrote:


Hi Artem,
I think you make a good point which is worth further consideration. If
any of the existing methods is really ripe for a change here, it’s the
send() that actually caused the problem. If that can be fixed so there are
no situations in which a lurking error breaks a transaction, that might be
the best.

Thanks,
Andrew


On 21 Jun 2024, at 01:51, Artem Livshits 

wrote:



I thought we still wait for requests (and their errors) to come in and

could handle fatal errors appropriately.

We do wait for requests, but my understanding is that when
commitTransaction("ignore send errors") we want to ignore errors.  So if

we

do

1. send
2. commitTransaction("ignore send errors")

the commit will succeed.  You can look at the example in
https://issues.apache.org/jira/browse/KAFKA-9279 and just substitute
commitTransaction with commitTransaction("ignore send errors") and we get
the buggy behavior back :-).  Actually, this would potentially be even
worse than the original buggy behavior because the bug was that we

ignored

errors that happened in the "send()" method itself, not necessarily the
ones that we got from the broker.

Actually, looking at https://github.com/apache/kafka/pull/11508/files,
wouldn't a better solution be to just throw the error from the "send"
method itself, rather than trying to set it to be thrown during commit?
This way the example in https://issues.apache.org/jira/browse/KAFKA-9279
would be fixed, and at the same time it would give an opportunity for KS

to

catch the error and ignore it if needed.  Not sure if we need a KIP for
that, just do a better fix of the old bug.

-Artem

On Thu, Jun 20, 2024 at 4:58 PM Justine Olshan



wrote:


I'm a bit late to the party, but the discussion here looks reasonable.
Moving the logic to a transactional method makes sense to me and makes

me

feel a bit better about keeping the complexity in the methods relevant

to

the issue.


One minor concern is that if we set "ignore send

errors" (or whatever we decide to name it) option without explicit

flush,

it'll actually lead to broken behavior as the application won't be able

to

stop a commit from proceeding even on fatal errors.

Is this with respect to the case a request is still inflight when we

call

commitTransaction? I thought we still wait for requests (and their

errors)

to come in and could handle fatal errors appropriately.

Justine

On Thu, Jun 20, 2024 at 4:32 PM Artem Livshits
 wrote:


Hi Matthias (and other folks who suggested ideas),


maybe `commitTransaction(CommitOptions)` or similar could be a good

way

forward?

I like this approach.  One minor concern is that if we set "ignore send
errors" (or whatever we decide to name it) option without explicit

flush,

it'll actually lead to broken behavior as the application won't be able

to

stop a commit from procee

Re: [DISCUSS] KIP-1059: Enable the Producer flush() method to clear the latest send() error

2024-06-21 Thread Alieh Saeedi
Hi all,


It is very exciting to see all the experts here raising very good points.

As we go further, we see more and more options to improve our solution,
which makes concluding and updating the KIP impossible.


The main suggestions so far are:

1. `flush` with `flushOptions` as input parameter

2. `commitTx` with `commitOptions` as input parameter

3. `send` must throw the exception


My concern about the 3rd suggestion:

1. Does the change cause any issue with backward compatibility?

2. The `send (bad record)` already transits the transaction to the error
state. No user, including Streams is able to transit the transaction back
from the error state. Do you mean we remove the
`maybeTransitionToErrorState(e)` from here

as well?

Cheers,
Alieh


On Fri, Jun 21, 2024 at 8:45 AM Andrew Schofield 
wrote:

> Hi Artem,
> I think you make a good point which is worth further consideration. If
> any of the existing methods is really ripe for a change here, it’s the
> send() that actually caused the problem. If that can be fixed so there are
> no situations in which a lurking error breaks a transaction, that might be
> the best.
>
> Thanks,
> Andrew
>
> > On 21 Jun 2024, at 01:51, Artem Livshits 
> wrote:
> >
> >> I thought we still wait for requests (and their errors) to come in and
> > could handle fatal errors appropriately.
> >
> > We do wait for requests, but my understanding is that when
> > commitTransaction("ignore send errors") we want to ignore errors.  So if
> we
> > do
> >
> > 1. send
> > 2. commitTransaction("ignore send errors")
> >
> > the commit will succeed.  You can look at the example in
> > https://issues.apache.org/jira/browse/KAFKA-9279 and just substitute
> > commitTransaction with commitTransaction("ignore send errors") and we get
> > the buggy behavior back :-).  Actually, this would potentially be even
> > worse than the original buggy behavior because the bug was that we
> ignored
> > errors that happened in the "send()" method itself, not necessarily the
> > ones that we got from the broker.
> >
> > Actually, looking at https://github.com/apache/kafka/pull/11508/files,
> > wouldn't a better solution be to just throw the error from the "send"
> > method itself, rather than trying to set it to be thrown during commit?
> > This way the example in https://issues.apache.org/jira/browse/KAFKA-9279
> > would be fixed, and at the same time it would give an opportunity for KS
> to
> > catch the error and ignore it if needed.  Not sure if we need a KIP for
> > that, just do a better fix of the old bug.
> >
> > -Artem
> >
> > On Thu, Jun 20, 2024 at 4:58 PM Justine Olshan
> 
> > wrote:
> >
> >> I'm a bit late to the party, but the discussion here looks reasonable.
> >> Moving the logic to a transactional method makes sense to me and makes
> me
> >> feel a bit better about keeping the complexity in the methods relevant
> to
> >> the issue.
> >>
> >>> One minor concern is that if we set "ignore send
> >> errors" (or whatever we decide to name it) option without explicit
> flush,
> >> it'll actually lead to broken behavior as the application won't be able
> to
> >> stop a commit from proceeding even on fatal errors.
> >>
> >> Is this with respect to the case a request is still inflight when we
> call
> >> commitTransaction? I thought we still wait for requests (and their
> errors)
> >> to come in and could handle fatal errors appropriately.
> >>
> >> Justine
> >>
> >> On Thu, Jun 20, 2024 at 4:32 PM Artem Livshits
> >>  wrote:
> >>
> >>> Hi Matthias (and other folks who suggested ideas),
> >>>
>  maybe `commitTransaction(CommitOptions)` or similar could be a good
> >> way
> >>> forward?
> >>>
> >>> I like this approach.  One minor concern is that if we set "ignore send
> >>> errors" (or whatever we decide to name it) option without explicit
> flush,
> >>> it'll actually lead to broken behavior as the application won't be able
> >> to
> >>> stop a commit from proceeding even on fatal errors.  But I guess we'll
> >> just
> >>> have to clearly document it.
> >>>
> >>> In some way we are basically adding a flag to optionally restore the
> >>> https://issues.apache.org/jira/browse/KAFKA-9279 bug, which is the
> >>> motivation for all these changes, anyway :-).
> >>>
> >>> -Artem
> >>>
> >>>
> >>> On Thu, Jun 20, 2024 at 2:18 PM Matthias J. Sax 
> >> wrote:
> >>>
>  Seems the option to use a config does not get a lot of support.
> 
>  So we need to go with some form or "overload / new method". I think
>  Chris' point about not coupling it to `flush()` but rather
>  `commitTransaction()` is actually a very good one; for non-tx case,
> the
>  different flush variants would not make sense.
> 
>  I also like Lianet's idea to pass in some "options" object, so maybe
>  `commitTransaction(CommitOptions)` or simila

Re: [DISCUSS] KIP-1059: Enable the Producer flush() method to clear the latest send() error

2024-06-20 Thread Andrew Schofield
Hi Artem,
I think you make a good point which is worth further consideration. If
any of the existing methods is really ripe for a change here, it’s the
send() that actually caused the problem. If that can be fixed so there are
no situations in which a lurking error breaks a transaction, that might be
the best.

Thanks,
Andrew

> On 21 Jun 2024, at 01:51, Artem Livshits  
> wrote:
>
>> I thought we still wait for requests (and their errors) to come in and
> could handle fatal errors appropriately.
>
> We do wait for requests, but my understanding is that when
> commitTransaction("ignore send errors") we want to ignore errors.  So if we
> do
>
> 1. send
> 2. commitTransaction("ignore send errors")
>
> the commit will succeed.  You can look at the example in
> https://issues.apache.org/jira/browse/KAFKA-9279 and just substitute
> commitTransaction with commitTransaction("ignore send errors") and we get
> the buggy behavior back :-).  Actually, this would potentially be even
> worse than the original buggy behavior because the bug was that we ignored
> errors that happened in the "send()" method itself, not necessarily the
> ones that we got from the broker.
>
> Actually, looking at https://github.com/apache/kafka/pull/11508/files,
> wouldn't a better solution be to just throw the error from the "send"
> method itself, rather than trying to set it to be thrown during commit?
> This way the example in https://issues.apache.org/jira/browse/KAFKA-9279
> would be fixed, and at the same time it would give an opportunity for KS to
> catch the error and ignore it if needed.  Not sure if we need a KIP for
> that, just do a better fix of the old bug.
>
> -Artem
>
> On Thu, Jun 20, 2024 at 4:58 PM Justine Olshan 
> wrote:
>
>> I'm a bit late to the party, but the discussion here looks reasonable.
>> Moving the logic to a transactional method makes sense to me and makes me
>> feel a bit better about keeping the complexity in the methods relevant to
>> the issue.
>>
>>> One minor concern is that if we set "ignore send
>> errors" (or whatever we decide to name it) option without explicit flush,
>> it'll actually lead to broken behavior as the application won't be able to
>> stop a commit from proceeding even on fatal errors.
>>
>> Is this with respect to the case a request is still inflight when we call
>> commitTransaction? I thought we still wait for requests (and their errors)
>> to come in and could handle fatal errors appropriately.
>>
>> Justine
>>
>> On Thu, Jun 20, 2024 at 4:32 PM Artem Livshits
>>  wrote:
>>
>>> Hi Matthias (and other folks who suggested ideas),
>>>
 maybe `commitTransaction(CommitOptions)` or similar could be a good
>> way
>>> forward?
>>>
>>> I like this approach.  One minor concern is that if we set "ignore send
>>> errors" (or whatever we decide to name it) option without explicit flush,
>>> it'll actually lead to broken behavior as the application won't be able
>> to
>>> stop a commit from proceeding even on fatal errors.  But I guess we'll
>> just
>>> have to clearly document it.
>>>
>>> In some way we are basically adding a flag to optionally restore the
>>> https://issues.apache.org/jira/browse/KAFKA-9279 bug, which is the
>>> motivation for all these changes, anyway :-).
>>>
>>> -Artem
>>>
>>>
>>> On Thu, Jun 20, 2024 at 2:18 PM Matthias J. Sax 
>> wrote:
>>>
 Seems the option to use a config does not get a lot of support.

 So we need to go with some form or "overload / new method". I think
 Chris' point about not coupling it to `flush()` but rather
 `commitTransaction()` is actually a very good one; for non-tx case, the
 different flush variants would not make sense.

 I also like Lianet's idea to pass in some "options" object, so maybe
 `commitTransaction(CommitOptions)` or similar could be a good way
 forward? It's much better than a `boolean` parameter, aesthetically, as
 we as extendable in the future if necessary.

 Given that we would pass in an optional parameter, we might not even
 need to deprecate the existing `commitTransaction()` method?



 -Matthias

 On 6/20/24 9:12 AM, Andrew Schofield wrote:
> Hi Alieh,
> Thanks for the KIP.
>
> I *really* don’t like adding a config which changes the behaviour of
>>> the
> flush() method. We already have too many configs. But I totally
 understand
> the problem that you’re trying to solve and some of the other
>>> suggestions
> in this thread seem neater.
>
> Personally, I would add another method to KafkaProducer. Not an
>>> overload
> on flush() because this is not flush() at all. Using Matthias’s
>>> options,
> I prefer (3).
>
> Thanks,
> Andrew
>
>> On 20 Jun 2024, at 15:08, Lianet M.  wrote:
>>
>> Hi all, thanks for the KIP Alieh!
>>
>> LM1. Totally agree with Artem's point about the config not being the
 most
>> explicit/flexible way to express this capabili

Re: [DISCUSS] KIP-1059: Enable the Producer flush() method to clear the latest send() error

2024-06-20 Thread Artem Livshits
>  I thought we still wait for requests (and their errors) to come in and
could handle fatal errors appropriately.

We do wait for requests, but my understanding is that when
commitTransaction("ignore send errors") we want to ignore errors.  So if we
do

1. send
2. commitTransaction("ignore send errors")

the commit will succeed.  You can look at the example in
https://issues.apache.org/jira/browse/KAFKA-9279 and just substitute
commitTransaction with commitTransaction("ignore send errors") and we get
the buggy behavior back :-).  Actually, this would potentially be even
worse than the original buggy behavior because the bug was that we ignored
errors that happened in the "send()" method itself, not necessarily the
ones that we got from the broker.

Actually, looking at https://github.com/apache/kafka/pull/11508/files,
wouldn't a better solution be to just throw the error from the "send"
method itself, rather than trying to set it to be thrown during commit?
This way the example in https://issues.apache.org/jira/browse/KAFKA-9279
would be fixed, and at the same time it would give an opportunity for KS to
catch the error and ignore it if needed.  Not sure if we need a KIP for
that, just do a better fix of the old bug.

-Artem

On Thu, Jun 20, 2024 at 4:58 PM Justine Olshan 
wrote:

> I'm a bit late to the party, but the discussion here looks reasonable.
> Moving the logic to a transactional method makes sense to me and makes me
> feel a bit better about keeping the complexity in the methods relevant to
> the issue.
>
> > One minor concern is that if we set "ignore send
> errors" (or whatever we decide to name it) option without explicit flush,
> it'll actually lead to broken behavior as the application won't be able to
> stop a commit from proceeding even on fatal errors.
>
> Is this with respect to the case a request is still inflight when we call
> commitTransaction? I thought we still wait for requests (and their errors)
> to come in and could handle fatal errors appropriately.
>
> Justine
>
> On Thu, Jun 20, 2024 at 4:32 PM Artem Livshits
>  wrote:
>
> > Hi Matthias (and other folks who suggested ideas),
> >
> > >  maybe `commitTransaction(CommitOptions)` or similar could be a good
> way
> > forward?
> >
> > I like this approach.  One minor concern is that if we set "ignore send
> > errors" (or whatever we decide to name it) option without explicit flush,
> > it'll actually lead to broken behavior as the application won't be able
> to
> > stop a commit from proceeding even on fatal errors.  But I guess we'll
> just
> > have to clearly document it.
> >
> > In some way we are basically adding a flag to optionally restore the
> > https://issues.apache.org/jira/browse/KAFKA-9279 bug, which is the
> > motivation for all these changes, anyway :-).
> >
> > -Artem
> >
> >
> > On Thu, Jun 20, 2024 at 2:18 PM Matthias J. Sax 
> wrote:
> >
> > > Seems the option to use a config does not get a lot of support.
> > >
> > > So we need to go with some form or "overload / new method". I think
> > > Chris' point about not coupling it to `flush()` but rather
> > > `commitTransaction()` is actually a very good one; for non-tx case, the
> > > different flush variants would not make sense.
> > >
> > > I also like Lianet's idea to pass in some "options" object, so maybe
> > > `commitTransaction(CommitOptions)` or similar could be a good way
> > > forward? It's much better than a `boolean` parameter, aesthetically, as
> > > we as extendable in the future if necessary.
> > >
> > > Given that we would pass in an optional parameter, we might not even
> > > need to deprecate the existing `commitTransaction()` method?
> > >
> > >
> > >
> > > -Matthias
> > >
> > > On 6/20/24 9:12 AM, Andrew Schofield wrote:
> > > > Hi Alieh,
> > > > Thanks for the KIP.
> > > >
> > > > I *really* don’t like adding a config which changes the behaviour of
> > the
> > > > flush() method. We already have too many configs. But I totally
> > > understand
> > > > the problem that you’re trying to solve and some of the other
> > suggestions
> > > > in this thread seem neater.
> > > >
> > > > Personally, I would add another method to KafkaProducer. Not an
> > overload
> > > > on flush() because this is not flush() at all. Using Matthias’s
> > options,
> > > > I prefer (3).
> > > >
> > > > Thanks,
> > > > Andrew
> > > >
> > > >> On 20 Jun 2024, at 15:08, Lianet M.  wrote:
> > > >>
> > > >> Hi all, thanks for the KIP Alieh!
> > > >>
> > > >> LM1. Totally agree with Artem's point about the config not being the
> > > most
> > > >> explicit/flexible way to express this capability. Getting then to
> > > Matthias
> > > >> 4 options, what I don't like about 3 and 4 is that it seems they
> might
> > > not
> > > >> age very well? Aren't we going to be wanting some other twist to the
> > > flush
> > > >> semantics that will have us adding yet another param to it, or
> another
> > > >> overloaded method? I truly don't have the context to answer that,
> but
> > > if it
> 

Re: [DISCUSS] KIP-1059: Enable the Producer flush() method to clear the latest send() error

2024-06-20 Thread Justine Olshan
I'm a bit late to the party, but the discussion here looks reasonable.
Moving the logic to a transactional method makes sense to me and makes me
feel a bit better about keeping the complexity in the methods relevant to
the issue.

> One minor concern is that if we set "ignore send
errors" (or whatever we decide to name it) option without explicit flush,
it'll actually lead to broken behavior as the application won't be able to
stop a commit from proceeding even on fatal errors.

Is this with respect to the case a request is still inflight when we call
commitTransaction? I thought we still wait for requests (and their errors)
to come in and could handle fatal errors appropriately.

Justine

On Thu, Jun 20, 2024 at 4:32 PM Artem Livshits
 wrote:

> Hi Matthias (and other folks who suggested ideas),
>
> >  maybe `commitTransaction(CommitOptions)` or similar could be a good way
> forward?
>
> I like this approach.  One minor concern is that if we set "ignore send
> errors" (or whatever we decide to name it) option without explicit flush,
> it'll actually lead to broken behavior as the application won't be able to
> stop a commit from proceeding even on fatal errors.  But I guess we'll just
> have to clearly document it.
>
> In some way we are basically adding a flag to optionally restore the
> https://issues.apache.org/jira/browse/KAFKA-9279 bug, which is the
> motivation for all these changes, anyway :-).
>
> -Artem
>
>
> On Thu, Jun 20, 2024 at 2:18 PM Matthias J. Sax  wrote:
>
> > Seems the option to use a config does not get a lot of support.
> >
> > So we need to go with some form or "overload / new method". I think
> > Chris' point about not coupling it to `flush()` but rather
> > `commitTransaction()` is actually a very good one; for non-tx case, the
> > different flush variants would not make sense.
> >
> > I also like Lianet's idea to pass in some "options" object, so maybe
> > `commitTransaction(CommitOptions)` or similar could be a good way
> > forward? It's much better than a `boolean` parameter, aesthetically, as
> > we as extendable in the future if necessary.
> >
> > Given that we would pass in an optional parameter, we might not even
> > need to deprecate the existing `commitTransaction()` method?
> >
> >
> >
> > -Matthias
> >
> > On 6/20/24 9:12 AM, Andrew Schofield wrote:
> > > Hi Alieh,
> > > Thanks for the KIP.
> > >
> > > I *really* don’t like adding a config which changes the behaviour of
> the
> > > flush() method. We already have too many configs. But I totally
> > understand
> > > the problem that you’re trying to solve and some of the other
> suggestions
> > > in this thread seem neater.
> > >
> > > Personally, I would add another method to KafkaProducer. Not an
> overload
> > > on flush() because this is not flush() at all. Using Matthias’s
> options,
> > > I prefer (3).
> > >
> > > Thanks,
> > > Andrew
> > >
> > >> On 20 Jun 2024, at 15:08, Lianet M.  wrote:
> > >>
> > >> Hi all, thanks for the KIP Alieh!
> > >>
> > >> LM1. Totally agree with Artem's point about the config not being the
> > most
> > >> explicit/flexible way to express this capability. Getting then to
> > Matthias
> > >> 4 options, what I don't like about 3 and 4 is that it seems they might
> > not
> > >> age very well? Aren't we going to be wanting some other twist to the
> > flush
> > >> semantics that will have us adding yet another param to it, or another
> > >> overloaded method? I truly don't have the context to answer that, but
> > if it
> > >> feels like a realistic future maybe adding some kind FlushOptions
> > params to
> > >> the flush would be better from an extensibility point of view. It
> would
> > >> only have the clearErrors option available for now but could accept
> any
> > >> other we may need. I find that this would remove the "ugliness"
> Matthias
> > >> pointed out for 3. and 4.
> > >>
> > >> LM2. No matter how we end up expressing the different semantics for
> > flush,
> > >> let's make sure we update the KIP on the flush and commitTransaction
> > java
> > >> docs. It currently states that  flush "clears the last exception" and
> > >> commitTransaction "will NOT throw" if called after flush, but it
> really
> > all
> > >> depends on the config/options/method used.
> > >>
> > >> LM3. I find it would be helpful to include an example to show the new
> > flow
> > >> that we're unblocking (I see this as the great gain here): flush with
> > clear
> > >> error option enabled -> catch and do whatever error handling we want
> ->
> > >> commitTransaction successfully
> > >>
> > >> Thanks!
> > >>
> > >> Lianet
> > >>
> > >> On Wed, Jun 19, 2024 at 11:26 PM Chris Egerton <
> fearthecel...@gmail.com
> > >
> > >> wrote:
> > >>
> > >>> Hi Matthias,
> > >>>
> > >>> I like the alternatives you've listed. One more that might help is
> if,
> > >>> instead of overloading flush(), we overloaded commitTransaction() to
> > >>> something like commitTransaction(boolean tolerateRecordErrors). This
> > seems
> > >>> sli

Re: [DISCUSS] KIP-1059: Enable the Producer flush() method to clear the latest send() error

2024-06-20 Thread Artem Livshits
Hi Matthias (and other folks who suggested ideas),

>  maybe `commitTransaction(CommitOptions)` or similar could be a good way
forward?

I like this approach.  One minor concern is that if we set "ignore send
errors" (or whatever we decide to name it) option without explicit flush,
it'll actually lead to broken behavior as the application won't be able to
stop a commit from proceeding even on fatal errors.  But I guess we'll just
have to clearly document it.

In some way we are basically adding a flag to optionally restore the
https://issues.apache.org/jira/browse/KAFKA-9279 bug, which is the
motivation for all these changes, anyway :-).

-Artem


On Thu, Jun 20, 2024 at 2:18 PM Matthias J. Sax  wrote:

> Seems the option to use a config does not get a lot of support.
>
> So we need to go with some form or "overload / new method". I think
> Chris' point about not coupling it to `flush()` but rather
> `commitTransaction()` is actually a very good one; for non-tx case, the
> different flush variants would not make sense.
>
> I also like Lianet's idea to pass in some "options" object, so maybe
> `commitTransaction(CommitOptions)` or similar could be a good way
> forward? It's much better than a `boolean` parameter, aesthetically, as
> we as extendable in the future if necessary.
>
> Given that we would pass in an optional parameter, we might not even
> need to deprecate the existing `commitTransaction()` method?
>
>
>
> -Matthias
>
> On 6/20/24 9:12 AM, Andrew Schofield wrote:
> > Hi Alieh,
> > Thanks for the KIP.
> >
> > I *really* don’t like adding a config which changes the behaviour of the
> > flush() method. We already have too many configs. But I totally
> understand
> > the problem that you’re trying to solve and some of the other suggestions
> > in this thread seem neater.
> >
> > Personally, I would add another method to KafkaProducer. Not an overload
> > on flush() because this is not flush() at all. Using Matthias’s options,
> > I prefer (3).
> >
> > Thanks,
> > Andrew
> >
> >> On 20 Jun 2024, at 15:08, Lianet M.  wrote:
> >>
> >> Hi all, thanks for the KIP Alieh!
> >>
> >> LM1. Totally agree with Artem's point about the config not being the
> most
> >> explicit/flexible way to express this capability. Getting then to
> Matthias
> >> 4 options, what I don't like about 3 and 4 is that it seems they might
> not
> >> age very well? Aren't we going to be wanting some other twist to the
> flush
> >> semantics that will have us adding yet another param to it, or another
> >> overloaded method? I truly don't have the context to answer that, but
> if it
> >> feels like a realistic future maybe adding some kind FlushOptions
> params to
> >> the flush would be better from an extensibility point of view. It would
> >> only have the clearErrors option available for now but could accept any
> >> other we may need. I find that this would remove the "ugliness" Matthias
> >> pointed out for 3. and 4.
> >>
> >> LM2. No matter how we end up expressing the different semantics for
> flush,
> >> let's make sure we update the KIP on the flush and commitTransaction
> java
> >> docs. It currently states that  flush "clears the last exception" and
> >> commitTransaction "will NOT throw" if called after flush, but it really
> all
> >> depends on the config/options/method used.
> >>
> >> LM3. I find it would be helpful to include an example to show the new
> flow
> >> that we're unblocking (I see this as the great gain here): flush with
> clear
> >> error option enabled -> catch and do whatever error handling we want ->
> >> commitTransaction successfully
> >>
> >> Thanks!
> >>
> >> Lianet
> >>
> >> On Wed, Jun 19, 2024 at 11:26 PM Chris Egerton  >
> >> wrote:
> >>
> >>> Hi Matthias,
> >>>
> >>> I like the alternatives you've listed. One more that might help is if,
> >>> instead of overloading flush(), we overloaded commitTransaction() to
> >>> something like commitTransaction(boolean tolerateRecordErrors). This
> seems
> >>> slightly cleaner in that it takes the behavioral change we want, which
> only
> >>> applies to transactional producers, to an API method that is only used
> for
> >>> transactional producers. It would also avoid the issue of whether or
> not
> >>> flush() (or a new variant of it with altered semantics) should throw or
> >>> not. Thoughts?
> >>>
> >>> Hi Alieh,
> >>>
> >>> Thanks for the KIP, I like this direction a lot more than the pluggable
> >>> handler!
> >>>
> >>> I share Artem's concerns that enabling this behavior via configuration
> >>> doesn't seem like a great fit. It's likely that application code will
> be
> >>> written in a style that only works with one type of behavior from
> >>> transactional producers, so requiring that application code to declare
> its
> >>> expectations for the behavior of its producer seems more appropriate
> than,
> >>> e.g., allowing users deploying that application to tweak a
> configuration
> >>> file that gets fed to producers spun up inside it.
> >>>
> >>> Chee

Re: [DISCUSS] KIP-1059: Enable the Producer flush() method to clear the latest send() error

2024-06-20 Thread Matthias J. Sax

Seems the option to use a config does not get a lot of support.

So we need to go with some form or "overload / new method". I think 
Chris' point about not coupling it to `flush()` but rather 
`commitTransaction()` is actually a very good one; for non-tx case, the 
different flush variants would not make sense.


I also like Lianet's idea to pass in some "options" object, so maybe 
`commitTransaction(CommitOptions)` or similar could be a good way 
forward? It's much better than a `boolean` parameter, aesthetically, as 
we as extendable in the future if necessary.


Given that we would pass in an optional parameter, we might not even 
need to deprecate the existing `commitTransaction()` method?




-Matthias

On 6/20/24 9:12 AM, Andrew Schofield wrote:

Hi Alieh,
Thanks for the KIP.

I *really* don’t like adding a config which changes the behaviour of the
flush() method. We already have too many configs. But I totally understand
the problem that you’re trying to solve and some of the other suggestions
in this thread seem neater.

Personally, I would add another method to KafkaProducer. Not an overload
on flush() because this is not flush() at all. Using Matthias’s options,
I prefer (3).

Thanks,
Andrew


On 20 Jun 2024, at 15:08, Lianet M.  wrote:

Hi all, thanks for the KIP Alieh!

LM1. Totally agree with Artem's point about the config not being the most
explicit/flexible way to express this capability. Getting then to Matthias
4 options, what I don't like about 3 and 4 is that it seems they might not
age very well? Aren't we going to be wanting some other twist to the flush
semantics that will have us adding yet another param to it, or another
overloaded method? I truly don't have the context to answer that, but if it
feels like a realistic future maybe adding some kind FlushOptions params to
the flush would be better from an extensibility point of view. It would
only have the clearErrors option available for now but could accept any
other we may need. I find that this would remove the "ugliness" Matthias
pointed out for 3. and 4.

LM2. No matter how we end up expressing the different semantics for flush,
let's make sure we update the KIP on the flush and commitTransaction java
docs. It currently states that  flush "clears the last exception" and
commitTransaction "will NOT throw" if called after flush, but it really all
depends on the config/options/method used.

LM3. I find it would be helpful to include an example to show the new flow
that we're unblocking (I see this as the great gain here): flush with clear
error option enabled -> catch and do whatever error handling we want ->
commitTransaction successfully

Thanks!

Lianet

On Wed, Jun 19, 2024 at 11:26 PM Chris Egerton 
wrote:


Hi Matthias,

I like the alternatives you've listed. One more that might help is if,
instead of overloading flush(), we overloaded commitTransaction() to
something like commitTransaction(boolean tolerateRecordErrors). This seems
slightly cleaner in that it takes the behavioral change we want, which only
applies to transactional producers, to an API method that is only used for
transactional producers. It would also avoid the issue of whether or not
flush() (or a new variant of it with altered semantics) should throw or
not. Thoughts?

Hi Alieh,

Thanks for the KIP, I like this direction a lot more than the pluggable
handler!

I share Artem's concerns that enabling this behavior via configuration
doesn't seem like a great fit. It's likely that application code will be
written in a style that only works with one type of behavior from
transactional producers, so requiring that application code to declare its
expectations for the behavior of its producer seems more appropriate than,
e.g., allowing users deploying that application to tweak a configuration
file that gets fed to producers spun up inside it.

Cheers,

Chris

On Wed, Jun 19, 2024 at 10:32 PM Matthias J. Sax  wrote:


Thanks for the KIP Alieh. I actually like the KIP as-is, but think
Arthem raises very good points...

Seems we have four options on how to move forward?

  1. add config to allow "silent error clearance" as the KIP proposes
  2. change flush() to clear error and let it throw
  3. add new flushAndThrow()` (or better name) which clears error and
throws
  4. add `flush(boolean clearAndThrow)` and let user pick (and deprecate
existing `flush()`)

For (2), given that it would be a behavior change, we might also need a
public "feature flag" config.

It seems, both (1) and (2) have the issue Artem mentioned. (3) and (4)
would be safer to this end, however, for both we kinda get an ugly API?

Not sure right now if I have any preference. Seems we need to pick some
evil and that there is no clear best solution? Would be good to her from
others what they think


-Matthias


On 6/18/24 8:39 PM, Artem Livshits wrote:

Hi Alieh,

Thank you for the KIP.  I have a couple of suggestions:

AL1.  We should throw an error from flush after we clear it.  This

would

make

Re: [DISCUSS] KIP-1059: Enable the Producer flush() method to clear the latest send() error

2024-06-20 Thread Andrew Schofield
Hi Alieh,
Thanks for the KIP.

I *really* don’t like adding a config which changes the behaviour of the
flush() method. We already have too many configs. But I totally understand
the problem that you’re trying to solve and some of the other suggestions
in this thread seem neater.

Personally, I would add another method to KafkaProducer. Not an overload
on flush() because this is not flush() at all. Using Matthias’s options,
I prefer (3).

Thanks,
Andrew

> On 20 Jun 2024, at 15:08, Lianet M.  wrote:
>
> Hi all, thanks for the KIP Alieh!
>
> LM1. Totally agree with Artem's point about the config not being the most
> explicit/flexible way to express this capability. Getting then to Matthias
> 4 options, what I don't like about 3 and 4 is that it seems they might not
> age very well? Aren't we going to be wanting some other twist to the flush
> semantics that will have us adding yet another param to it, or another
> overloaded method? I truly don't have the context to answer that, but if it
> feels like a realistic future maybe adding some kind FlushOptions params to
> the flush would be better from an extensibility point of view. It would
> only have the clearErrors option available for now but could accept any
> other we may need. I find that this would remove the "ugliness" Matthias
> pointed out for 3. and 4.
>
> LM2. No matter how we end up expressing the different semantics for flush,
> let's make sure we update the KIP on the flush and commitTransaction java
> docs. It currently states that  flush "clears the last exception" and
> commitTransaction "will NOT throw" if called after flush, but it really all
> depends on the config/options/method used.
>
> LM3. I find it would be helpful to include an example to show the new flow
> that we're unblocking (I see this as the great gain here): flush with clear
> error option enabled -> catch and do whatever error handling we want ->
> commitTransaction successfully
>
> Thanks!
>
> Lianet
>
> On Wed, Jun 19, 2024 at 11:26 PM Chris Egerton 
> wrote:
>
>> Hi Matthias,
>>
>> I like the alternatives you've listed. One more that might help is if,
>> instead of overloading flush(), we overloaded commitTransaction() to
>> something like commitTransaction(boolean tolerateRecordErrors). This seems
>> slightly cleaner in that it takes the behavioral change we want, which only
>> applies to transactional producers, to an API method that is only used for
>> transactional producers. It would also avoid the issue of whether or not
>> flush() (or a new variant of it with altered semantics) should throw or
>> not. Thoughts?
>>
>> Hi Alieh,
>>
>> Thanks for the KIP, I like this direction a lot more than the pluggable
>> handler!
>>
>> I share Artem's concerns that enabling this behavior via configuration
>> doesn't seem like a great fit. It's likely that application code will be
>> written in a style that only works with one type of behavior from
>> transactional producers, so requiring that application code to declare its
>> expectations for the behavior of its producer seems more appropriate than,
>> e.g., allowing users deploying that application to tweak a configuration
>> file that gets fed to producers spun up inside it.
>>
>> Cheers,
>>
>> Chris
>>
>> On Wed, Jun 19, 2024 at 10:32 PM Matthias J. Sax  wrote:
>>
>>> Thanks for the KIP Alieh. I actually like the KIP as-is, but think
>>> Arthem raises very good points...
>>>
>>> Seems we have four options on how to move forward?
>>>
>>>  1. add config to allow "silent error clearance" as the KIP proposes
>>>  2. change flush() to clear error and let it throw
>>>  3. add new flushAndThrow()` (or better name) which clears error and
>>> throws
>>>  4. add `flush(boolean clearAndThrow)` and let user pick (and deprecate
>>> existing `flush()`)
>>>
>>> For (2), given that it would be a behavior change, we might also need a
>>> public "feature flag" config.
>>>
>>> It seems, both (1) and (2) have the issue Artem mentioned. (3) and (4)
>>> would be safer to this end, however, for both we kinda get an ugly API?
>>>
>>> Not sure right now if I have any preference. Seems we need to pick some
>>> evil and that there is no clear best solution? Would be good to her from
>>> others what they think
>>>
>>>
>>> -Matthias
>>>
>>>
>>> On 6/18/24 8:39 PM, Artem Livshits wrote:
 Hi Alieh,

 Thank you for the KIP.  I have a couple of suggestions:

 AL1.  We should throw an error from flush after we clear it.  This
>> would
 make it so that both "send + commit" and "send + flush + commit" (the
 latter looks like just a more verbose way to express the former, and it
 would be intuitive if it behaves the same) would throw if the
>> transaction
 has an error (so if the code is written either way it's going be
>>> correct).
 At the same time, the latter could be extended by the caller to
>> intercept
 exceptions from flush, ignore as needed, and commit the transaction.
>>> This
 solution would keep basi

Re: [DISCUSS] KIP-1059: Enable the Producer flush() method to clear the latest send() error

2024-06-20 Thread Lianet M.
Hi all, thanks for the KIP Alieh!

LM1. Totally agree with Artem's point about the config not being the most
explicit/flexible way to express this capability. Getting then to Matthias
4 options, what I don't like about 3 and 4 is that it seems they might not
age very well? Aren't we going to be wanting some other twist to the flush
semantics that will have us adding yet another param to it, or another
overloaded method? I truly don't have the context to answer that, but if it
feels like a realistic future maybe adding some kind FlushOptions params to
the flush would be better from an extensibility point of view. It would
only have the clearErrors option available for now but could accept any
other we may need. I find that this would remove the "ugliness" Matthias
pointed out for 3. and 4.

LM2. No matter how we end up expressing the different semantics for flush,
let's make sure we update the KIP on the flush and commitTransaction java
docs. It currently states that  flush "clears the last exception" and
commitTransaction "will NOT throw" if called after flush, but it really all
depends on the config/options/method used.

LM3. I find it would be helpful to include an example to show the new flow
that we're unblocking (I see this as the great gain here): flush with clear
error option enabled -> catch and do whatever error handling we want ->
commitTransaction successfully

Thanks!

Lianet

On Wed, Jun 19, 2024 at 11:26 PM Chris Egerton 
wrote:

> Hi Matthias,
>
> I like the alternatives you've listed. One more that might help is if,
> instead of overloading flush(), we overloaded commitTransaction() to
> something like commitTransaction(boolean tolerateRecordErrors). This seems
> slightly cleaner in that it takes the behavioral change we want, which only
> applies to transactional producers, to an API method that is only used for
> transactional producers. It would also avoid the issue of whether or not
> flush() (or a new variant of it with altered semantics) should throw or
> not. Thoughts?
>
> Hi Alieh,
>
> Thanks for the KIP, I like this direction a lot more than the pluggable
> handler!
>
> I share Artem's concerns that enabling this behavior via configuration
> doesn't seem like a great fit. It's likely that application code will be
> written in a style that only works with one type of behavior from
> transactional producers, so requiring that application code to declare its
> expectations for the behavior of its producer seems more appropriate than,
> e.g., allowing users deploying that application to tweak a configuration
> file that gets fed to producers spun up inside it.
>
> Cheers,
>
> Chris
>
> On Wed, Jun 19, 2024 at 10:32 PM Matthias J. Sax  wrote:
>
> > Thanks for the KIP Alieh. I actually like the KIP as-is, but think
> > Arthem raises very good points...
> >
> > Seems we have four options on how to move forward?
> >
> >   1. add config to allow "silent error clearance" as the KIP proposes
> >   2. change flush() to clear error and let it throw
> >   3. add new flushAndThrow()` (or better name) which clears error and
> > throws
> >   4. add `flush(boolean clearAndThrow)` and let user pick (and deprecate
> > existing `flush()`)
> >
> > For (2), given that it would be a behavior change, we might also need a
> > public "feature flag" config.
> >
> > It seems, both (1) and (2) have the issue Artem mentioned. (3) and (4)
> > would be safer to this end, however, for both we kinda get an ugly API?
> >
> > Not sure right now if I have any preference. Seems we need to pick some
> > evil and that there is no clear best solution? Would be good to her from
> > others what they think
> >
> >
> > -Matthias
> >
> >
> > On 6/18/24 8:39 PM, Artem Livshits wrote:
> > > Hi Alieh,
> > >
> > > Thank you for the KIP.  I have a couple of suggestions:
> > >
> > > AL1.  We should throw an error from flush after we clear it.  This
> would
> > > make it so that both "send + commit" and "send + flush + commit" (the
> > > latter looks like just a more verbose way to express the former, and it
> > > would be intuitive if it behaves the same) would throw if the
> transaction
> > > has an error (so if the code is written either way it's going be
> > correct).
> > > At the same time, the latter could be extended by the caller to
> intercept
> > > exceptions from flush, ignore as needed, and commit the transaction.
> > This
> > > solution would keep basic things simple (if someone has code that
> doesn't
> > > require advanced error handling, then basic "send + flush + commit"
> would
> > > do the right thing) and advanced things possible, an application can
> add
> > > try + catch around flush and ignore some errors.
> > >
> > > AL2.  I'm not sure if config is the best way to express the
> modification
> > of
> > > the "flush" semantics -- the application logic that calls "flush" needs
> > to
> > > match the "flush" semantics and configuring semantics in a detached
> place
> > > creates a room for bugs due to discrepancies

Re: [DISCUSS] KIP-1059: Enable the Producer flush() method to clear the latest send() error

2024-06-19 Thread Chris Egerton
Hi Matthias,

I like the alternatives you've listed. One more that might help is if,
instead of overloading flush(), we overloaded commitTransaction() to
something like commitTransaction(boolean tolerateRecordErrors). This seems
slightly cleaner in that it takes the behavioral change we want, which only
applies to transactional producers, to an API method that is only used for
transactional producers. It would also avoid the issue of whether or not
flush() (or a new variant of it with altered semantics) should throw or
not. Thoughts?

Hi Alieh,

Thanks for the KIP, I like this direction a lot more than the pluggable
handler!

I share Artem's concerns that enabling this behavior via configuration
doesn't seem like a great fit. It's likely that application code will be
written in a style that only works with one type of behavior from
transactional producers, so requiring that application code to declare its
expectations for the behavior of its producer seems more appropriate than,
e.g., allowing users deploying that application to tweak a configuration
file that gets fed to producers spun up inside it.

Cheers,

Chris

On Wed, Jun 19, 2024 at 10:32 PM Matthias J. Sax  wrote:

> Thanks for the KIP Alieh. I actually like the KIP as-is, but think
> Arthem raises very good points...
>
> Seems we have four options on how to move forward?
>
>   1. add config to allow "silent error clearance" as the KIP proposes
>   2. change flush() to clear error and let it throw
>   3. add new flushAndThrow()` (or better name) which clears error and
> throws
>   4. add `flush(boolean clearAndThrow)` and let user pick (and deprecate
> existing `flush()`)
>
> For (2), given that it would be a behavior change, we might also need a
> public "feature flag" config.
>
> It seems, both (1) and (2) have the issue Artem mentioned. (3) and (4)
> would be safer to this end, however, for both we kinda get an ugly API?
>
> Not sure right now if I have any preference. Seems we need to pick some
> evil and that there is no clear best solution? Would be good to her from
> others what they think
>
>
> -Matthias
>
>
> On 6/18/24 8:39 PM, Artem Livshits wrote:
> > Hi Alieh,
> >
> > Thank you for the KIP.  I have a couple of suggestions:
> >
> > AL1.  We should throw an error from flush after we clear it.  This would
> > make it so that both "send + commit" and "send + flush + commit" (the
> > latter looks like just a more verbose way to express the former, and it
> > would be intuitive if it behaves the same) would throw if the transaction
> > has an error (so if the code is written either way it's going be
> correct).
> > At the same time, the latter could be extended by the caller to intercept
> > exceptions from flush, ignore as needed, and commit the transaction.
> This
> > solution would keep basic things simple (if someone has code that doesn't
> > require advanced error handling, then basic "send + flush + commit" would
> > do the right thing) and advanced things possible, an application can add
> > try + catch around flush and ignore some errors.
> >
> > AL2.  I'm not sure if config is the best way to express the modification
> of
> > the "flush" semantics -- the application logic that calls "flush" needs
> to
> > match the "flush" semantics and configuring semantics in a detached place
> > creates a room for bugs due to discrepancies.  This can be especially bad
> > if the producer loads configuration from a file at run time, in that
> case a
> > mistake in configuration could break the application because it was
> written
> > to expect one "flush" semantics but the semantics is switched.  Given
> that
> > the "flush" semantics needs to match the caller's expectation, a way to
> > accomplish that would be to pass the caller's expectation to the "flush"
> > call by either have a method with a different name or have an overload
> with
> > a Boolen flag that would configure the semantics (the current method
> could
> > just redirect to the new one).
> >
> > -Artem
> >
> > On Mon, Jun 17, 2024 at 9:09 AM Alieh Saeedi
> 
> > wrote:
> >
> >> Hi all,
> >>
> >> I'd like to kick off a discussion for KIP-1059 that suggests adding a
> new
> >> feature to the Producer flush() method.
> >>
> >>
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-1059%3A+Enable+the+Producer+flush%28%29+method+to+clear+the+latest+send%28%29+error
> >>
> >> Cheers,
> >> Alieh
> >>
> >
>


Re: [DISCUSS] KIP-1059: Enable the Producer flush() method to clear the latest send() error

2024-06-19 Thread Matthias J. Sax
Thanks for the KIP Alieh. I actually like the KIP as-is, but think 
Arthem raises very good points...


Seems we have four options on how to move forward?

 1. add config to allow "silent error clearance" as the KIP proposes
 2. change flush() to clear error and let it throw
 3. add new flushAndThrow()` (or better name) which clears error and throws
 4. add `flush(boolean clearAndThrow)` and let user pick (and deprecate 
existing `flush()`)


For (2), given that it would be a behavior change, we might also need a 
public "feature flag" config.


It seems, both (1) and (2) have the issue Artem mentioned. (3) and (4) 
would be safer to this end, however, for both we kinda get an ugly API?


Not sure right now if I have any preference. Seems we need to pick some 
evil and that there is no clear best solution? Would be good to her from 
others what they think



-Matthias


On 6/18/24 8:39 PM, Artem Livshits wrote:

Hi Alieh,

Thank you for the KIP.  I have a couple of suggestions:

AL1.  We should throw an error from flush after we clear it.  This would
make it so that both "send + commit" and "send + flush + commit" (the
latter looks like just a more verbose way to express the former, and it
would be intuitive if it behaves the same) would throw if the transaction
has an error (so if the code is written either way it's going be correct).
At the same time, the latter could be extended by the caller to intercept
exceptions from flush, ignore as needed, and commit the transaction.  This
solution would keep basic things simple (if someone has code that doesn't
require advanced error handling, then basic "send + flush + commit" would
do the right thing) and advanced things possible, an application can add
try + catch around flush and ignore some errors.

AL2.  I'm not sure if config is the best way to express the modification of
the "flush" semantics -- the application logic that calls "flush" needs to
match the "flush" semantics and configuring semantics in a detached place
creates a room for bugs due to discrepancies.  This can be especially bad
if the producer loads configuration from a file at run time, in that case a
mistake in configuration could break the application because it was written
to expect one "flush" semantics but the semantics is switched.  Given that
the "flush" semantics needs to match the caller's expectation, a way to
accomplish that would be to pass the caller's expectation to the "flush"
call by either have a method with a different name or have an overload with
a Boolen flag that would configure the semantics (the current method could
just redirect to the new one).

-Artem

On Mon, Jun 17, 2024 at 9:09 AM Alieh Saeedi 
wrote:


Hi all,

I'd like to kick off a discussion for KIP-1059 that suggests adding a new
feature to the Producer flush() method.


https://cwiki.apache.org/confluence/display/KAFKA/KIP-1059%3A+Enable+the+Producer+flush%28%29+method+to+clear+the+latest+send%28%29+error

Cheers,
Alieh





Re: [DISCUSS] KIP-1059: Enable the Producer flush() method to clear the latest send() error

2024-06-18 Thread Artem Livshits
Hi Alieh,

Thank you for the KIP.  I have a couple of suggestions:

AL1.  We should throw an error from flush after we clear it.  This would
make it so that both "send + commit" and "send + flush + commit" (the
latter looks like just a more verbose way to express the former, and it
would be intuitive if it behaves the same) would throw if the transaction
has an error (so if the code is written either way it's going be correct).
At the same time, the latter could be extended by the caller to intercept
exceptions from flush, ignore as needed, and commit the transaction.  This
solution would keep basic things simple (if someone has code that doesn't
require advanced error handling, then basic "send + flush + commit" would
do the right thing) and advanced things possible, an application can add
try + catch around flush and ignore some errors.

AL2.  I'm not sure if config is the best way to express the modification of
the "flush" semantics -- the application logic that calls "flush" needs to
match the "flush" semantics and configuring semantics in a detached place
creates a room for bugs due to discrepancies.  This can be especially bad
if the producer loads configuration from a file at run time, in that case a
mistake in configuration could break the application because it was written
to expect one "flush" semantics but the semantics is switched.  Given that
the "flush" semantics needs to match the caller's expectation, a way to
accomplish that would be to pass the caller's expectation to the "flush"
call by either have a method with a different name or have an overload with
a Boolen flag that would configure the semantics (the current method could
just redirect to the new one).

-Artem

On Mon, Jun 17, 2024 at 9:09 AM Alieh Saeedi 
wrote:

> Hi all,
>
> I'd like to kick off a discussion for KIP-1059 that suggests adding a new
> feature to the Producer flush() method.
>
>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-1059%3A+Enable+the+Producer+flush%28%29+method+to+clear+the+latest+send%28%29+error
>
> Cheers,
> Alieh
>


[DISCUSS] KIP-1059: Enable the Producer flush() method to clear the latest send() error

2024-06-17 Thread Alieh Saeedi
Hi all,

I'd like to kick off a discussion for KIP-1059 that suggests adding a new
feature to the Producer flush() method.

https://cwiki.apache.org/confluence/display/KAFKA/KIP-1059%3A+Enable+the+Producer+flush%28%29+method+to+clear+the+latest+send%28%29+error

Cheers,
Alieh