confluentinc / confluent-kafka-dotnet

Confluent's Apache Kafka .NET client
https://github.com/confluentinc/confluent-kafka-dotnet/wiki
Apache License 2.0
53 stars 857 forks source link

OutOfOrderSequenceException #1943

Open sgajjarhawk opened 1 year ago

sgajjarhawk commented 1 year ago

Description

Hi,

When sending data to kafka using an async, idempotent producer, we receive errors showing:

%3|1668238507.285|ERROR|<PUBLISHER_APP_NAME>#producer-1|: Fatal error: Broker: Broker received an out of order sequence number: ProduceRequest for <TOPIC_NAME> [38] with 1 message(s) failed due to sequence desynchronization with broker 1 (PID{Id:22181,Epoch:0}, base seq 0, idemp state change 8184098ms ago, last partition error NO_ERROR (actions , base seq 0..0, base msgid 0, -1ms ago)

Our Producer configuration looks like:

config = new ProducerConfig { BootstrapServers = appConfiguration.KafkaBootStrapServers, SecurityProtocol = SecurityProtocol.Ssl, Acks = Acks.All, EnableIdempotence = true, ClientId = appConfiguration.KafkaClientID };

Our Producer method call looks like:

try { Headers headers = new Headers(); headers.Add(<CONSTANT>.ATT_TRANSACTION_ID, Encoding.ASCII.GetBytes(transactionId)); DeliveryResult<string, string> response = await kafkaProducer.ProduceAsync(topic, new Message<string, string> { Key = key, Value = eventData, Headers = headers }); return response; // logging } catch (ProduceException<string, string> ex) { // catch exception, processing and logging return null; } catch (Exception ex) { // catch exception and logging return null; }

How to reproduce

  1. Start the Kafka cluster (v2.8.0) with 3 Kafka brokers and
  2. Start a .Net producer producing messages to Kafka topic using Confluent.Kafka 1.7.0 with librdkafka.redist v1.7.0, which keep sending messages
  3. Take out broker-1 for a patching and add that back to the cluster while producer is still producing messages.
  4. After broker-1 is added back, the application throws exception ".......ProduceRequest for [38] with 1 message(s) failed due to sequence with broker 1 (PID{Id:22181,Epoch:0}, base seq 0, idemp state change 8184098ms ago, last partition error NO_ERROR (actions , base seq 0..0, base msgid 0, -1ms ago)........."
  5. Need to restart the producer application to remediate the issue. This is causing highly business transactional messages to be lost

Checklist

Please provide the following information:

2022-11-12T01:13:41.000-06:00[2022-11-12 07:13:39,294] INFO [ReplicaFetcherManager on broker 1] shutting down (kafka.server.ReplicaFetcherManager) 2022-11-12T01:13:41.000-06:00[2022-11-12 07:13:39,294] INFO [LogDirFailureHandler]: Shutdown completed (kafka.server.ReplicaManager$LogDirFailureHandler) 2022-11-12T01:17:16.000-06:00[2022-11-12 07:17:08,961] INFO Registered signal handlers for TERM, INT, HUP (org.apache.kafka.common.utils.LoggingSignalHandler) 2022-11-12T01:17:16.000-06:00Copy[2022-11-12 07:17:08,967] INFO starting (kafka.server.KafkaServer)[2022-11-12 07:17:08,967] INFO starting (kafka.server.KafkaServer)

mhowlett commented 1 year ago

@emasab @edenhill

we were seeing this in the recent transactional producer issue. in the transactional producer case, the recent fix (not yet in a release) allows the producer to recover by aborting the transaction, but if you're just using the idempotent producer, the fix does not address this case (i believe). note that we did not see this issue with the java client in testing.

sgajjarhawk commented 1 year ago

Are you referring to aborting the producer transactional handler & re-initialize the handler to produce messages using Confluent-kafka .net client? Could you please clarify?

Also, could you confirm if there's an existing/known issue with confluent-kafka-dotnet client library??

mhowlett commented 1 year ago

this is a known issue.

my original comment was a bit confusing, i updated it a bit.

sgajjarhawk commented 1 year ago

this is a known issue.

my original comment was a bit confusing, i updated it a bit.

Thank you!

milindl commented 1 year ago

Hi @sgajjarhawk,

Right now, indeed the only thing you can do is destroying, and then recreating your producer whenever you encounter a fatal error like this one, since a fatal error is be indicative of cases when the idempotence guarantee might no longer be working.

I've tried reproducing the issue using a few long-running tests on the latest version of client, 1.9.2, and broker version 2.8, but I'm not having any luck.

If you can reproduce the issue consistently, can you provide the complete debug logs for the client in question? You would need to add { ... Debug = "all" } to your config for this. It would be best if you can use the new version of the dotnet client (1.9.2) for this - I don't think this particular issue is addressed, but there are still a bunch of changes to the idempotent/transactional producer, and it would be better to have logs from the latest version.

a-elsheikh commented 1 year ago

hello friends - I've recently experienced this issue myself during a temporary network outage, only a few clients were affected but they were stuck in a perpetual loop with the following errors:

client A lib version - 1.8.2

Confluent.Kafka.ProduceException``2[System.Byte[],System.Byte[]]: ProduceRequest for TOPIC [0] with 3 message(s) failed due to sequence desynchronization with broker 1003 (PID{Id:38056,Epoch:0}, base seq 853718, idemp state change 40282872ms ago, last partition error _TRANSPORT (actions Refresh,MsgPossiblyPersisted, base seq 852855..852858, base msgid 852856, 72860ms ago) at Confluent.Kafka.Producer``2.Produce(TopicPartition topicPartition, Message``2 message, Action``1 deliveryHandler)

client B lib version - 2.0.2

Confluent.Kafka.ProduceException``2[System.Byte[],System.Byte[]]: Broker: Broker received an out of order sequence number at Confluent.Kafka.Producer``2.Produce(TopicPartition topicPartition, Message``2 message, Action``1 deliveryHandler)

anchitj commented 6 months ago

Hi, is this still an issue with the latest version?

emasab commented 3 months ago

@sgajjarhawk @anchitj is asking because there was this fix in 2.3.0

After a possibly persisted error, such as a disconnection or a timeout, next expected sequence used to increase, leading to a fatal error if the message wasn't persisted and the second one in queue failed with an OUT_OF_ORDER_SEQUENCE_NUMBER. The error could contain the message "sequence desynchronization" with just one possibly persisted error or "rewound sequence number" in case of multiple errored messages. Solved by treating the possible persisted message as not persisted, and expecting a DUPLICATE_SEQUENCE_NUMBER error in case it was or NO_ERROR in case it wasn't, in both cases the message will be considered delivered (https://github.com/confluentinc/librdkafka/pull/4438).