Hi Becket,
I have read kafka source code and found that the error won't be propagated
to client if the list of
topic-partition is empty [1], because it bind the error with each
topic-partition. If this list is empty,
then that error won't be packaged into response body. That made the client
didn't
Hi,
I found that the source code [1] in kafka showed that it always check if
`newPartitionsInTransaction`
is empty before calling `enqueueRequest(addPartitionsToTransactionHandler())`,
that is not
applied to flink kafka producer code [2].
I wrote a simple producer with the `flushNewPartitions` co
Hi,
Trying to dig out why `Error.NOT_COORDINATOR` happened in broker, I opened
flink's log level to DEBUG for producer. And I found some logs from flink
side
regarding this error. Below is some log snippet.
It seems that producer client didn't catch this error and retry to find new
coordinator.
T
Hi Becket,
I found that those transactions were tend to be failed
with InvalidTxnStateException if
they never sent any records but committed after some brokers being
restarted.
Because the error state transition always failed from EMPTY to COMMIT, I
run a
job with only one parallelism with or wit
Hi Becket,
One more thing, I have tried to restart other brokers without active
controller, but
this exception might happen as well. So it should be independent of the
active
controller like you said.
Best,
Tony Wei
Tony Wei 於 2019年9月18日 週三 下午6:14寫道:
> Hi Becket,
>
> I have reproduced this pr
Hi Becket,
I have reproduced this problem in our development environment. Below is the
log message with debug level.
Seems that the exception was from broker-3, and I also found other error
code in broker-2 during the time.
There are others INVALID_TXN_STATE error for other transaction id. I just
Hi Tony,
>From the symptom it is not quite clear to me what may cause this issue.
Supposedly the TransactionCoordinator is independent of the active
controller, so bouncing the active controller should not have special
impact on the transactions (at least not every time). If this is stably
reprodu
Hi,
Has anyone run into the same problem? I have updated my producer
transaction timeout to 1.5 hours,
but the problem sill happened when I restarted broker with active
controller. It might not due to the
problem that checkpoint duration is too long causing transaction timeout. I
had no more clue
Hi Tony,
I'm sorry I cannot help you with this issue, but Becket (in CC) might have
an idea what went wrong here.
Best, Fabian
Am Mi., 14. Aug. 2019 um 07:00 Uhr schrieb Tony Wei :
> Hi,
>
> Currently, I was trying to update our kafka cluster with larger `
> transaction.max.timeout.ms`. The
> o
Hi,
Currently, I was trying to update our kafka cluster with larger `
transaction.max.timeout.ms`. The
original setting is kafka's default value (i.e. 15 minutes) and I tried to
set as 3 hours.
When I was doing rolling-restart for my brokers, this exception came to me
on the next checkpoint
after
10 matches
Mail list logo